You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2017/08/22 20:55:08 UTC

[01/11] cassandra git commit: move streaming to use netty

Repository: cassandra
Updated Branches:
  refs/heads/trunk 3d4a7e7b6 -> fc92db2b9


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index ceaa4d1..68c6034 100644
--- a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
@@ -19,103 +19,64 @@ package org.apache.cassandra.streaming.messages;
 
 import java.io.IOException;
 import java.net.InetAddress;
-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.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.DataOutputBufferFixed;
-import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.StreamSession;
 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 class StreamInitMessage extends StreamMessage
 {
-    public static IVersionedSerializer<StreamInitMessage> serializer = new StreamInitMessageSerializer();
+    public static Serializer<StreamInitMessage> serializer = new StreamInitMessageSerializer();
 
     public final InetAddress from;
     public final int sessionIndex;
     public final UUID planId;
     public final StreamOperation streamOperation;
 
-    // true if this init message is to connect for outgoing message on receiving side
-    public final boolean isForOutgoing;
     public final boolean keepSSTableLevel;
     public final UUID pendingRepair;
     public final PreviewKind previewKind;
 
-    public StreamInitMessage(InetAddress from, int sessionIndex, UUID planId, StreamOperation streamOperation, boolean isForOutgoing, boolean keepSSTableLevel, UUID pendingRepair, PreviewKind previewKind)
+    public StreamInitMessage(InetAddress from, int sessionIndex, UUID planId, StreamOperation streamOperation, boolean keepSSTableLevel, UUID pendingRepair, PreviewKind previewKind)
     {
+        super(Type.STREAM_INIT);
         this.from = from;
         this.sessionIndex = sessionIndex;
         this.planId = planId;
         this.streamOperation = streamOperation;
-        this.isForOutgoing = isForOutgoing;
         this.keepSSTableLevel = keepSSTableLevel;
         this.pendingRepair = pendingRepair;
         this.previewKind = previewKind;
     }
 
-    /**
-     * 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)
+    @Override
+    public String toString()
     {
-        int header = 0;
-        // set compression bit.
-        if (compress)
-            header |= 4;
-        // set streaming bit
-        header |= 8;
-        // Setting up the version bit
-        header |= (version << 8);
-
-        byte[] bytes;
-        try
-        {
-            int size = (int)StreamInitMessage.serializer.serializedSize(this, version);
-            try (DataOutputBuffer buffer = new DataOutputBufferFixed(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;
+        StringBuilder sb = new StringBuilder(128);
+        sb.append("StreamInitMessage: from = ").append(from);
+        sb.append(", planId = ").append(planId).append(", session index = ").append(sessionIndex);
+        return sb.toString();
     }
 
-    private static class StreamInitMessageSerializer implements IVersionedSerializer<StreamInitMessage>
+    private static class StreamInitMessageSerializer implements Serializer<StreamInitMessage>
     {
-        public void serialize(StreamInitMessage message, DataOutputPlus out, int version) throws IOException
+        public void serialize(StreamInitMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
         {
             CompactEndpointSerializationHelper.serialize(message.from, out);
             out.writeInt(message.sessionIndex);
             UUIDSerializer.serializer.serialize(message.planId, out, MessagingService.current_version);
             out.writeUTF(message.streamOperation.getDescription());
-            out.writeBoolean(message.isForOutgoing);
             out.writeBoolean(message.keepSSTableLevel);
 
             out.writeBoolean(message.pendingRepair != null);
@@ -126,18 +87,17 @@ public class StreamInitMessage
             out.writeInt(message.previewKind.getSerializationVal());
         }
 
-        public StreamInitMessage deserialize(DataInputPlus in, int version) throws IOException
+        public StreamInitMessage deserialize(DataInputPlus in, int version, StreamSession session) throws IOException
         {
             InetAddress from = CompactEndpointSerializationHelper.deserialize(in);
             int sessionIndex = in.readInt();
             UUID planId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
             String description = in.readUTF();
-            boolean sentByInitiator = in.readBoolean();
             boolean keepSSTableLevel = in.readBoolean();
 
             UUID pendingRepair = in.readBoolean() ? UUIDSerializer.serializer.deserialize(in, version) : null;
             PreviewKind previewKind = PreviewKind.deserialize(in.readInt());
-            return new StreamInitMessage(from, sessionIndex, planId, StreamOperation.fromString(description), sentByInitiator, keepSSTableLevel, pendingRepair, previewKind);
+            return new StreamInitMessage(from, sessionIndex, planId, StreamOperation.fromString(description), keepSSTableLevel, pendingRepair, previewKind);
         }
 
         public long serializedSize(StreamInitMessage message, int version)
@@ -146,7 +106,6 @@ public class StreamInitMessage
             size += TypeSizes.sizeof(message.sessionIndex);
             size += UUIDSerializer.serializer.serializedSize(message.planId, MessagingService.current_version);
             size += TypeSizes.sizeof(message.streamOperation.getDescription());
-            size += TypeSizes.sizeof(message.isForOutgoing);
             size += TypeSizes.sizeof(message.keepSSTableLevel);
             size += TypeSizes.sizeof(message.pendingRepair != null);
             if (message.pendingRepair != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 48def64..feeab05 100644
--- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
@@ -18,10 +18,8 @@
 package org.apache.cassandra.streaming.messages;
 
 import java.io.IOException;
-import java.net.SocketException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
 
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
@@ -36,67 +34,47 @@ public abstract class StreamMessage
     public static final int VERSION_40 = 5;
     public static final int CURRENT_VERSION = VERSION_40;
 
-    private transient volatile boolean sent = false;
-
     public static void serialize(StreamMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
     {
-        ByteBuffer buff = ByteBuffer.allocate(1);
         // message type
-        buff.put(message.type.type);
-        buff.flip();
-        out.write(buff);
+        out.writeByte(message.type.type);
         message.type.outSerializer.serialize(message, out, version, session);
     }
 
-    public static StreamMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
-    {
-        ByteBuffer buff = ByteBuffer.allocate(1);
-        int readBytes = in.read(buff);
-        if (readBytes > 0)
-        {
-            buff.flip();
-            Type type = Type.get(buff.get());
-            return type.inSerializer.deserialize(in, version, session);
-        }
-        else if (readBytes == 0)
-        {
-            // input socket buffer was not filled yet
-            return null;
-        }
-        else
-        {
-            // possibly socket gets closed
-            throw new SocketException("End-of-stream reached");
-        }
-    }
-
-    public void sent()
+    public static long serializedSize(StreamMessage message, int version) throws IOException
     {
-        sent = true;
+        return 1 + message.type.outSerializer.serializedSize(message, version);
     }
 
-    public boolean wasSent()
+    public static StreamMessage deserialize(DataInputPlus in, int version, StreamSession session) throws IOException
     {
-        return sent;
+        byte b = in.readByte();
+        if (b == 0)
+            b = -1;
+        Type type = Type.get(b);
+        return type.inSerializer.deserialize(in, version, session);
     }
 
     /** StreamMessage serializer */
     public static interface Serializer<V extends StreamMessage>
     {
-        V deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException;
+        V deserialize(DataInputPlus in, int version, StreamSession session) throws IOException;
         void serialize(V message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException;
+        long serializedSize(V message, int version) throws IOException;
     }
 
     /** StreamMessage types */
-    public static enum Type
+    public enum Type
     {
-        PREPARE(1, 5, PrepareMessage.serializer),
+        PREPARE_SYN(1, 5, PrepareSynMessage.serializer),
         FILE(2, 0, IncomingFileMessage.serializer, OutgoingFileMessage.serializer),
         RECEIVED(3, 4, ReceivedMessage.serializer),
-        RETRY(4, 4, RetryMessage.serializer),
         COMPLETE(5, 1, CompleteMessage.serializer),
         SESSION_FAILED(6, 5, SessionFailedMessage.serializer),
-        KEEP_ALIVE(7, 5, KeepAliveMessage.serializer);
+        KEEP_ALIVE(7, 5, KeepAliveMessage.serializer),
+        PREPARE_SYNACK(8, 5, PrepareSynAckMessage.serializer),
+        PREPARE_ACK(9, 5, PrepareAckMessage.serializer),
+        STREAM_INIT(10, 5, StreamInitMessage.serializer);
 
         public static Type get(byte type)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
index aa23f45..d119081 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
@@ -15,20 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.cassandra.tools;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.nio.channels.SocketChannel;
 
+import io.netty.channel.Channel;
 import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.security.SSLFactory;
+import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
 import org.apache.cassandra.streaming.StreamConnectionFactory;
-import org.apache.cassandra.utils.FBUtilities;
 
-public class BulkLoadConnectionFactory implements StreamConnectionFactory
+public class BulkLoadConnectionFactory extends DefaultConnectionFactory implements StreamConnectionFactory
 {
     private final boolean outboundBindAny;
     private final int storagePort;
@@ -43,24 +42,15 @@ public class BulkLoadConnectionFactory implements StreamConnectionFactory
         this.outboundBindAny = outboundBindAny;
     }
 
-    public Socket createConnection(InetAddress peer) throws IOException
+    public Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion) throws IOException
     {
         // Connect to secure port for all peers if ServerEncryptionOptions is configured other than 'none'
         // When 'all', 'dc' and 'rack', server nodes always have SSL port open, and since thin client like sstableloader
         // does not know which node is in which dc/rack, connecting to SSL port is always the option.
-        if (encryptionOptions != null && encryptionOptions.internode_encryption != EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none)
-        {
-            if (outboundBindAny)
-                return SSLFactory.getSocket(encryptionOptions, peer, secureStoragePort);
-            else
-                return SSLFactory.getSocket(encryptionOptions, peer, secureStoragePort, FBUtilities.getLocalAddress(), 0);
-        }
-        else
-        {
-            Socket socket = SocketChannel.open(new InetSocketAddress(peer, storagePort)).socket();
-            if (outboundBindAny && !socket.isBound())
-                socket.bind(new InetSocketAddress(FBUtilities.getLocalAddress(), 0));
-            return socket;
-        }
+        int port = encryptionOptions != null && encryptionOptions.internode_encryption != EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none ?
+                   secureStoragePort : storagePort;
+
+        connectionId = connectionId.withNewConnectionAddress(new InetSocketAddress(connectionId.remote(), port));
+        return createConnection(connectionId, protocolVersion, encryptionOptions);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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 109a51e..267a6d5 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -1076,8 +1076,6 @@ public class NodeProbe implements AutoCloseable
                 return ssProxy.getCasContentionTimeout();
             case "truncate":
                 return ssProxy.getTruncateRpcTimeout();
-            case "streamingsocket":
-                return ssProxy.getStreamingSocketTimeout();
             default:
                 throw new RuntimeException("Timeout type requires one of (" + GetTimeout.TIMEOUT_TYPES + ")");
         }
@@ -1156,11 +1154,6 @@ public class NodeProbe implements AutoCloseable
             case "truncate":
                 ssProxy.setTruncateRpcTimeout(value);
                 break;
-            case "streamingsocket":
-                if (value > Integer.MAX_VALUE)
-                    throw new RuntimeException("streamingsocket timeout must be less than " + Integer.MAX_VALUE);
-                ssProxy.setStreamingSocketTimeout((int) value);
-                break;
             default:
                 throw new RuntimeException("Timeout type requires one of (" + GetTimeout.TIMEOUT_TYPES + ")");
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java b/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
index 6c9b541..deac8a3 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
@@ -31,7 +31,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 @Command(name = "gettimeout", description = "Print the timeout of the given type in ms")
 public class GetTimeout extends NodeToolCmd
 {
-    public static final String TIMEOUT_TYPES = "read, range, write, counterwrite, cascontention, truncate, streamingsocket, misc (general rpc_timeout_in_ms)";
+    public static final String TIMEOUT_TYPES = "read, range, write, counterwrite, cascontention, truncate, misc (general rpc_timeout_in_ms)";
 
     @Arguments(usage = "<timeout_type>", description = "The timeout type, one of (" + TIMEOUT_TYPES + ")")
     private List<String> args = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/utils/UUIDGen.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index 8fac816..37f08bf 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -40,6 +40,8 @@ public class UUIDGen
     private static final long START_EPOCH = -12219292800000L;
     private static final long clockSeqAndNode = makeClockSeqAndNode();
 
+    public static final int UUID_LEN = 16;
+
     /*
      * The min and max possible lsb for a UUID.
      * Note that his is not 0 and all 1's because Cassandra TimeUUIDType
@@ -106,10 +108,10 @@ public class UUIDGen
     }
 
     /**
-     * Similar to {@link getTimeUUIDFromMicros}, but randomize (using SecureRandom) the clock and sequence.
+     * Similar to {@link #getTimeUUIDFromMicros}, but randomize (using SecureRandom) the clock and sequence.
      * <p>
      * If you can guarantee that the {@code whenInMicros} argument is unique (for this JVM instance) for
-     * every call, then you should prefer {@link getTimeUUIDFromMicros} which is faster. If you can't
+     * every call, then you should prefer {@link #getTimeUUIDFromMicros} which is faster. If you can't
      * guarantee this however, this method will ensure the returned UUID are still unique (accross calls)
      * through randomization.
      *
@@ -143,7 +145,7 @@ public class UUIDGen
 
     public static ByteBuffer toByteBuffer(UUID uuid)
     {
-        ByteBuffer buffer = ByteBuffer.allocate(16);
+        ByteBuffer buffer = ByteBuffer.allocate(UUID_LEN);
         buffer.putLong(uuid.getMostSignificantBits());
         buffer.putLong(uuid.getLeastSignificantBits());
         buffer.flip();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
index a30d6c9..799ac77 100644
--- a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
+++ b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
@@ -24,10 +24,12 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.io.Files;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -62,26 +64,42 @@ public class LongStreamingTest
     }
 
     @Test
-    public void testCompressedStream() throws InvalidRequestException, IOException, ExecutionException, InterruptedException
+    public void testSstableCompressionStreaming() throws InterruptedException, ExecutionException, IOException
     {
-        String KS = "cql_keyspace";
+        testStream(true);
+    }
+
+    @Test
+    public void testStreamCompressionStreaming() throws InterruptedException, ExecutionException, IOException
+    {
+        testStream(false);
+    }
+
+    private void testStream(boolean useSstableCompression) throws InvalidRequestException, IOException, ExecutionException, InterruptedException
+    {
+        String KS = useSstableCompression ? "sstable_compression_ks" : "stream_compression_ks";
         String TABLE = "table1";
 
         File tempdir = Files.createTempDir();
         File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
         assert dataDir.mkdirs();
 
-        String schema = "CREATE TABLE cql_keyspace.table1 ("
+        String schema = "CREATE TABLE " + KS + '.'  + TABLE + "  ("
                         + "  k int PRIMARY KEY,"
                         + "  v1 text,"
                         + "  v2 int"
-                        + ");";// with compression = {};";
-        String insert = "INSERT INTO cql_keyspace.table1 (k, v1, v2) VALUES (?, ?, ?)";
+                        + ") with compression = " + (useSstableCompression ? "{'class': 'LZ4Compressor'};" : "{};");
+        String insert = "INSERT INTO " + KS + '.'  + TABLE + " (k, v1, v2) VALUES (?, ?, ?)";
         CQLSSTableWriter writer = CQLSSTableWriter.builder()
                                                   .sorted()
                                                   .inDirectory(dataDir)
                                                   .forTable(schema)
                                                   .using(insert).build();
+
+        CompressionParams compressionParams = Keyspace.open(KS).getColumnFamilyStore(TABLE).metadata().params.compression;
+        Assert.assertEquals(useSstableCompression, compressionParams.isEnabled());
+
+
         long start = System.nanoTime();
 
         for (int i = 0; i < 10_000_000; i++)
@@ -103,7 +121,7 @@ public class LongStreamingTest
             private String ks;
             public void init(String keyspace)
             {
-                for (Range<Token> range : StorageService.instance.getLocalRanges("cql_keyspace"))
+                for (Range<Token> range : StorageService.instance.getLocalRanges(KS))
                     addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
 
                 this.ks = keyspace;
@@ -130,7 +148,7 @@ public class LongStreamingTest
             private String ks;
             public void init(String keyspace)
             {
-                for (Range<Token> range : StorageService.instance.getLocalRanges("cql_keyspace"))
+                for (Range<Token> range : StorageService.instance.getLocalRanges(KS))
                     addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
 
                 this.ks = keyspace;
@@ -160,7 +178,7 @@ public class LongStreamingTest
                                          millis / 1000d,
                                          (dataSize * 2 / (1 << 20) / (millis / 1000d)) * 8));
 
-        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM cql_keyspace.table1 limit 100;");
+        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM " + KS + '.'  + TABLE + " limit 100;");
         assertEquals(100, rs.size());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java b/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
index cd7e381..385ebb7 100644
--- a/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
@@ -54,7 +54,7 @@ public class PreparedStatementsTest extends SchemaLoader
 
         // Currently the native server start method return before the server is fully binded to the socket, so we need
         // to wait slightly before trying to connect to it. We should fix this but in the meantime using a sleep.
-        Thread.sleep(500);
+        Thread.sleep(1500);
 
         cluster = Cluster.builder().addContactPoint("127.0.0.1")
                                    .withPort(DatabaseDescriptor.getNativeTransportPort())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
index 175ab53..1803c51 100644
--- a/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
+++ b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
@@ -378,7 +378,7 @@ public class RewindableDataInputStreamPlusTest
             //finish reading again previous sequence
 
             reader.mark();
-            //read 3 bytes - OK
+            //read 3 bytes - START
             assertEquals('a', reader.readChar());
             //read 1 more bytes - CAPACITY will exhaust when trying to reset :(
             assertEquals(1, reader.readShort());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java b/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
index fa6e2b5..100e1e0 100644
--- a/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
+++ b/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
@@ -42,7 +42,7 @@ import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
 
-import static org.apache.cassandra.net.async.InboundHandshakeHandler.State.MESSAGING_HANDSHAKE_COMPLETE;
+import static org.apache.cassandra.net.async.InboundHandshakeHandler.State.HANDSHAKE_COMPLETE;
 import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.READY;
 
 public class HandshakeHandlersTest
@@ -100,7 +100,7 @@ public class HandshakeHandlersTest
             inboundChannel.writeInbound(o);
 
         Assert.assertEquals(READY, imc.getState());
-        Assert.assertEquals(MESSAGING_HANDSHAKE_COMPLETE, inboundHandshakeHandler.getState());
+        Assert.assertEquals(HANDSHAKE_COMPLETE, inboundHandshakeHandler.getState());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java b/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
index 44dc469..4d9829f 100644
--- a/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
+++ b/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
@@ -181,7 +181,7 @@ public class InboundHandshakeHandlerTest
         buf = new ThirdHandshakeMessage(MESSAGING_VERSION, addr.getAddress()).encode(PooledByteBufAllocator.DEFAULT);
         state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
 
-        Assert.assertEquals(State.MESSAGING_HANDSHAKE_COMPLETE, state);
+        Assert.assertEquals(State.HANDSHAKE_COMPLETE, state);
         Assert.assertTrue(channel.isOpen());
         Assert.assertTrue(channel.isActive());
         Assert.assertFalse(channel.outboundMessages().isEmpty());
@@ -217,7 +217,7 @@ public class InboundHandshakeHandlerTest
         buf.writeInt(MESSAGING_VERSION);
         CompactEndpointSerializationHelper.serialize(addr.getAddress(), new ByteBufOutputStream(buf));
         State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
-        Assert.assertEquals(State.MESSAGING_HANDSHAKE_COMPLETE, state);
+        Assert.assertEquals(State.HANDSHAKE_COMPLETE, state);
         Assert.assertTrue(channel.isOpen());
         Assert.assertTrue(channel.isActive());
     }
@@ -268,9 +268,9 @@ public class InboundHandshakeHandlerTest
         handler.setHandshakeTimeout(future);
         Assert.assertFalse(future.isCancelled());
         Assert.assertTrue(channel.isOpen());
-        handler.setState(State.MESSAGING_HANDSHAKE_COMPLETE);
+        handler.setState(State.HANDSHAKE_COMPLETE);
         handler.failHandshake(channel.pipeline().firstContext());
-        Assert.assertSame(State.MESSAGING_HANDSHAKE_COMPLETE, handler.getState());
+        Assert.assertSame(State.HANDSHAKE_COMPLETE, handler.getState());
         Assert.assertTrue(channel.isOpen());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
index 772e47d..d6dd633 100644
--- a/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
+++ b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
@@ -24,11 +24,7 @@ import java.net.InetSocketAddress;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
-import java.util.concurrent.Delayed;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.net.ssl.SSLHandshakeException;
 
@@ -475,45 +471,4 @@ public class OutboundMessagingConnectionTest
         Assert.assertNotSame(omc.getConnectionId(), originalId);
         Assert.assertSame(NOT_READY, omc.getState());
     }
-
-    private static class TestScheduledFuture implements ScheduledFuture<Object>
-    {
-        private boolean cancelled = false;
-
-        public long getDelay(TimeUnit unit)
-        {
-            return 0;
-        }
-
-        public int compareTo(Delayed o)
-        {
-            return 0;
-        }
-
-        public boolean cancel(boolean mayInterruptIfRunning)
-        {
-            cancelled = true;
-            return false;
-        }
-
-        public boolean isCancelled()
-        {
-            return cancelled;
-        }
-
-        public boolean isDone()
-        {
-            return false;
-        }
-
-        public Object get() throws InterruptedException, ExecutionException
-        {
-            return null;
-        }
-
-        public Object get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-        {
-            return null;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java b/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java
new file mode 100644
index 0000000..4968196
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.async;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.embedded.EmbeddedChannel;
+
+public class RebufferingByteBufDataInputPlusTest
+{
+    private EmbeddedChannel channel;
+    private RebufferingByteBufDataInputPlus inputPlus;
+    private ByteBuf buf;
+
+    @Before
+    public void setUp()
+    {
+        channel = new EmbeddedChannel();
+        inputPlus = new RebufferingByteBufDataInputPlus(1 << 10, 1 << 11, channel.config());
+    }
+
+    @After
+    public void tearDown()
+    {
+        inputPlus.close();
+        channel.close();
+
+        if (buf != null && buf.refCnt() > 0)
+            buf.release(buf.refCnt());
+    }
+
+    @Test (expected = IllegalArgumentException.class)
+    public void ctor_badWaterMarks()
+    {
+        inputPlus = new RebufferingByteBufDataInputPlus(2, 1, null);
+    }
+
+    @Test
+    public void isOpen()
+    {
+        Assert.assertTrue(inputPlus.isOpen());
+        inputPlus.markClose();
+        Assert.assertFalse(inputPlus.isOpen());
+    }
+
+    @Test (expected = IllegalStateException.class)
+    public void append_closed()
+    {
+        inputPlus.markClose();
+        buf = channel.alloc().buffer(4);
+        inputPlus.append(buf);
+    }
+
+    @Test
+    public void append_normal() throws EOFException
+    {
+        int size = 4;
+        buf = channel.alloc().buffer(size);
+        buf.writerIndex(size);
+        inputPlus.append(buf);
+        Assert.assertEquals(buf.readableBytes(), inputPlus.available());
+    }
+
+    @Test
+    public void read() throws IOException
+    {
+        // put two buffers of 8 bytes each into the queue.
+        // then read an int, then a long. the latter tests offset into the inputPlus, as well as spanning across queued buffers.
+        // the values of those int/long will both be '42', but spread across both queue buffers.
+        ByteBuf buf = channel.alloc().buffer(8);
+        buf.writeInt(42);
+        buf.writerIndex(8);
+        inputPlus.append(buf);
+        buf = channel.alloc().buffer(8);
+        buf.writeInt(42);
+        buf.writerIndex(8);
+        inputPlus.append(buf);
+        Assert.assertEquals(16, inputPlus.available());
+
+        ByteBuffer out = ByteBuffer.allocate(4);
+        int readCount = inputPlus.read(out);
+        Assert.assertEquals(4, readCount);
+        out.flip();
+        Assert.assertEquals(42, out.getInt());
+        Assert.assertEquals(12, inputPlus.available());
+
+        out = ByteBuffer.allocate(8);
+        readCount = inputPlus.read(out);
+        Assert.assertEquals(8, readCount);
+        out.flip();
+        Assert.assertEquals(42, out.getLong());
+        Assert.assertEquals(4, inputPlus.available());
+    }
+
+    @Test (expected = EOFException.class)
+    public void read_closed() throws IOException
+    {
+        inputPlus.markClose();
+        ByteBuffer buf = ByteBuffer.allocate(1);
+        inputPlus.read(buf);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/net/async/TestScheduledFuture.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/TestScheduledFuture.java b/test/unit/org/apache/cassandra/net/async/TestScheduledFuture.java
new file mode 100644
index 0000000..f5475ce
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/TestScheduledFuture.java
@@ -0,0 +1,66 @@
+/*
+ * 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.async;
+
+import java.util.concurrent.Delayed;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class TestScheduledFuture implements ScheduledFuture<Object>
+{
+    private boolean cancelled = false;
+
+    public long getDelay(TimeUnit unit)
+    {
+        return 0;
+    }
+
+    public int compareTo(Delayed o)
+    {
+        return 0;
+    }
+
+    public boolean cancel(boolean mayInterruptIfRunning)
+    {
+        cancelled = true;
+        return false;
+    }
+
+    public boolean isCancelled()
+    {
+        return cancelled;
+    }
+
+    public boolean isDone()
+    {
+        return false;
+    }
+
+    public Object get() throws InterruptedException, ExecutionException
+    {
+        return null;
+    }
+
+    public Object get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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 4efdb21..afc5b25 100644
--- a/test/unit/org/apache/cassandra/service/RemoveTest.java
+++ b/test/unit/org/apache/cassandra/service/RemoveTest.java
@@ -70,6 +70,7 @@ public class RemoveTest
     public static void setupClass() throws ConfigurationException
     {
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
+        MessagingService.instance().listen();
     }
 
     @AfterClass
@@ -86,7 +87,6 @@ public class RemoveTest
         // create a ring of 5 nodes
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 6);
 
-        MessagingService.instance().listen();
         removalhost = hosts.get(5);
         hosts.remove(removalhost);
         removalId = hostIds.get(5);
@@ -98,7 +98,6 @@ public class RemoveTest
     {
         MessagingService.instance().clearMessageSinks();
         MessagingService.instance().clearCallbacksUnsafe();
-        MessagingService.instance().shutdown();
     }
 
     @Test(expected = UnsupportedOperationException.class)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index 21c8375..5c29698 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -32,6 +32,7 @@ import org.junit.BeforeClass;
 import org.junit.After;
 import org.junit.Test;
 
+import io.netty.channel.embedded.EmbeddedChannel;
 import junit.framework.Assert;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -74,7 +75,7 @@ public class StreamTransferTaskTest
     public void testScheduleTimeout() throws Exception
     {
         InetAddress peer = FBUtilities.getBroadcastAddress();
-        StreamSession session = new StreamSession(peer, peer, null, 0, true, null, PreviewKind.NONE);
+        StreamSession session = new StreamSession(peer, peer, (connectionId, protocolVersion) -> new EmbeddedChannel(), 0, true, UUID.randomUUID(), PreviewKind.ALL);
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
 
         // create two sstables
@@ -120,7 +121,7 @@ public class StreamTransferTaskTest
     public void testFailSessionDuringTransferShouldNotReleaseReferences() throws Exception
     {
         InetAddress peer = FBUtilities.getBroadcastAddress();
-        StreamCoordinator streamCoordinator = new StreamCoordinator(1, true, null, false, null, PreviewKind.NONE);
+        StreamCoordinator streamCoordinator = new StreamCoordinator(1, true, new DefaultConnectionFactory(), false, null, PreviewKind.NONE);
         StreamResultFuture future = StreamResultFuture.init(UUID.randomUUID(), StreamOperation.OTHER, Collections.<StreamEventHandler>emptyList(), streamCoordinator);
         StreamSession session = new StreamSession(peer, peer, null, 0, true, null, PreviewKind.NONE);
         session.init(future);
@@ -159,7 +160,7 @@ public class StreamTransferTaskTest
         }
 
         //fail stream session mid-transfer
-        session.onError(new Exception("Fake exception"));
+        session.onError(new Exception("Fake exception")).get(5, TimeUnit.SECONDS);
 
         //make sure reference was not released
         for (Ref<SSTableReader> ref : refs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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 6a5002e..7a51d0c 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -26,7 +26,6 @@ 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.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.slf4j.Logger;
@@ -47,7 +46,6 @@ import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.StorageService;
@@ -62,9 +60,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-
-// TODO:JEB intentionally breaking this with CASSANDRA-8457 until CASSANDRA-12229
-@Ignore
 public class StreamingTransferTest
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamingTransferTest.class);
@@ -244,7 +239,6 @@ public class StreamingTransferTest
         ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key0"))));
         StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER).transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getTableName());
         streamPlan.execute().get();
-        verifyConnectionsAreClosed();
 
         //cannot add ranges after stream session is finished
         try
@@ -262,7 +256,6 @@ public class StreamingTransferTest
     {
         StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER).transferFiles(LOCAL, makeStreamingDetails(ranges, Refs.tryRef(Arrays.asList(sstable))));
         streamPlan.execute().get();
-        verifyConnectionsAreClosed();
 
         //cannot add files after stream session is finished
         try
@@ -276,27 +269,6 @@ public class StreamingTransferTest
         }
     }
 
-    /**
-     * Test that finished incoming connections are removed from MessagingService (CASSANDRA-11854)
-     */
-    private void verifyConnectionsAreClosed() throws InterruptedException
-    {
-        // TODO:JEB intentionally breaking this with CASSANDRA-8457 until CASSANDRA-12229
-        //after stream session is finished, message handlers may take several milliseconds to be closed
-//        outer:
-//        for (int i = 0; i <= 100; i++)
-//        {
-//            for (MessagingService.SocketThread socketThread : MessagingService.instance().getSocketThreads())
-//                if (!socketThread.connections.isEmpty())
-//                {
-//                    Thread.sleep(100);
-//                    continue outer;
-//                }
-//            return;
-//        }
-//        fail("Streaming connections remain registered in MessagingService");
-    }
-
     private Collection<StreamSession.SSTableStreamingSections> makeStreamingDetails(List<Range<Token>> ranges, Refs<SSTableReader> sstables)
     {
         ArrayList<StreamSession.SSTableStreamingSections> details = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java b/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java
new file mode 100644
index 0000000..a9849a3
--- /dev/null
+++ b/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java
@@ -0,0 +1,202 @@
+/*
+ * 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.async;
+
+import java.net.InetSocketAddress;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.async.TestScheduledFuture;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.messages.CompleteMessage;
+
+public class NettyStreamingMessageSenderTest
+{
+    private static final InetSocketAddress REMOTE_ADDR = new InetSocketAddress("127.0.0.2", 0);
+
+    private EmbeddedChannel channel;
+    private StreamSession session;
+    private NettyStreamingMessageSender sender;
+    private NettyStreamingMessageSender.FileStreamTask fileStreamTask;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Before
+    public void setUp()
+    {
+        channel = new EmbeddedChannel();
+        channel.attr(NettyStreamingMessageSender.TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
+        UUID pendingRepair = UUID.randomUUID();
+        session = new StreamSession(REMOTE_ADDR.getAddress(), REMOTE_ADDR.getAddress(), (connectionId, protocolVersion) -> null, 0, true, pendingRepair, PreviewKind.ALL);
+        StreamResultFuture future = StreamResultFuture.initReceivingSide(0, UUID.randomUUID(), StreamOperation.REPAIR, REMOTE_ADDR.getAddress(), channel, true, pendingRepair, session.getPreviewKind());
+        session.init(future);
+        sender = session.getMessageSender();
+        sender.setControlMessageChannel(channel);
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (fileStreamTask != null)
+            fileStreamTask.unsetChannel();
+    }
+
+    @Test
+    public void KeepAliveTask_normalSend()
+    {
+        Assert.assertTrue(channel.isOpen());
+        NettyStreamingMessageSender.KeepAliveTask task = sender.new KeepAliveTask(channel, session);
+        task.run();
+        Assert.assertTrue(channel.releaseOutbound());
+    }
+
+    @Test
+    public void KeepAliveTask_channelClosed()
+    {
+        channel.close();
+        Assert.assertFalse(channel.isOpen());
+        channel.releaseOutbound();
+        NettyStreamingMessageSender.KeepAliveTask task = sender.new KeepAliveTask(channel, session);
+        task.future = new TestScheduledFuture();
+        Assert.assertFalse(task.future.isCancelled());
+        task.run();
+        Assert.assertTrue(task.future.isCancelled());
+        Assert.assertFalse(channel.releaseOutbound());
+    }
+
+    @Test
+    public void KeepAliveTask_closed()
+    {
+        Assert.assertTrue(channel.isOpen());
+        NettyStreamingMessageSender.KeepAliveTask task = sender.new KeepAliveTask(channel, session);
+        task.future = new TestScheduledFuture();
+        Assert.assertFalse(task.future.isCancelled());
+
+        sender.setClosed();
+        Assert.assertFalse(sender.connected());
+        task.run();
+        Assert.assertTrue(task.future.isCancelled());
+        Assert.assertFalse(channel.releaseOutbound());
+    }
+
+    @Test
+    public void KeepAliveTask_CurrentlyStreaming()
+    {
+        Assert.assertTrue(channel.isOpen());
+        channel.attr(NettyStreamingMessageSender.TRANSFERRING_FILE_ATTR).set(Boolean.TRUE);
+        NettyStreamingMessageSender.KeepAliveTask task = sender.new KeepAliveTask(channel, session);
+        task.future = new TestScheduledFuture();
+        Assert.assertFalse(task.future.isCancelled());
+
+        Assert.assertTrue(sender.connected());
+        task.run();
+        Assert.assertFalse(task.future.isCancelled());
+        Assert.assertFalse(channel.releaseOutbound());
+    }
+
+    @Test
+    public void FileStreamTask_acquirePermit_closed()
+    {
+        fileStreamTask = sender.new FileStreamTask(null);
+        sender.setClosed();
+        Assert.assertFalse(fileStreamTask.acquirePermit(1));
+    }
+
+    @Test
+    public void FileStreamTask_acquirePermit_HapppyPath()
+    {
+        int permits = sender.semaphoreAvailablePermits();
+        fileStreamTask = sender.new FileStreamTask(null);
+        Assert.assertTrue(fileStreamTask.acquirePermit(1));
+        Assert.assertEquals(permits - 1, sender.semaphoreAvailablePermits());
+    }
+
+    @Test
+    public void FileStreamTask_BadChannelAttr()
+    {
+        int permits = sender.semaphoreAvailablePermits();
+        channel.attr(NettyStreamingMessageSender.TRANSFERRING_FILE_ATTR).set(Boolean.TRUE);
+        fileStreamTask = sender.new FileStreamTask(null);
+        fileStreamTask.injectChannel(channel);
+        fileStreamTask.run();
+        Assert.assertEquals(StreamSession.State.FAILED, session.state());
+        Assert.assertTrue(channel.releaseOutbound()); // when the session fails, it will send a SessionFailed msg
+        Assert.assertEquals(permits, sender.semaphoreAvailablePermits());
+    }
+
+    @Test
+    public void FileStreamTask_HappyPath()
+    {
+        int permits = sender.semaphoreAvailablePermits();
+        fileStreamTask = sender.new FileStreamTask(new CompleteMessage());
+        fileStreamTask.injectChannel(channel);
+        fileStreamTask.run();
+        Assert.assertNotEquals(StreamSession.State.FAILED, session.state());
+        Assert.assertTrue(channel.releaseOutbound());
+        Assert.assertEquals(permits, sender.semaphoreAvailablePermits());
+    }
+
+    @Test
+    public void onControlMessageComplete_HappyPath()
+    {
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertTrue(sender.connected());
+        ChannelPromise promise = channel.newPromise();
+        promise.setSuccess();
+        Assert.assertNull(sender.onControlMessageComplete(promise, new CompleteMessage()));
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertTrue(sender.connected());
+        Assert.assertNotEquals(StreamSession.State.FAILED, session.state());
+    }
+
+    @Test
+    public void onControlMessageComplete_Exception() throws InterruptedException, ExecutionException, TimeoutException
+    {
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertTrue(sender.connected());
+        ChannelPromise promise = channel.newPromise();
+        promise.setFailure(new RuntimeException("this is just a testing exception"));
+        Future f = sender.onControlMessageComplete(promise, new CompleteMessage());
+
+        f.get(5, TimeUnit.SECONDS);
+
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertFalse(sender.connected());
+        Assert.assertEquals(StreamSession.State.FAILED, session.state());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/streaming/async/StreamCompressionSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/async/StreamCompressionSerializerTest.java b/test/unit/org/apache/cassandra/streaming/async/StreamCompressionSerializerTest.java
new file mode 100644
index 0000000..fff7b17
--- /dev/null
+++ b/test/unit/org/apache/cassandra/streaming/async/StreamCompressionSerializerTest.java
@@ -0,0 +1,135 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.util.Random;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.lz4.LZ4FastDecompressor;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+
+public class StreamCompressionSerializerTest
+{
+    private static final int VERSION = StreamMessage.CURRENT_VERSION;
+    private static final Random random = new Random(2347623847623L);
+
+    private final ByteBufAllocator allocator = PooledByteBufAllocator.DEFAULT;
+    private final StreamCompressionSerializer serializer = new StreamCompressionSerializer(allocator);
+    private final LZ4Compressor compressor = LZ4Factory.fastestInstance().fastCompressor();
+    private final LZ4FastDecompressor decompressor = LZ4Factory.fastestInstance().fastDecompressor();
+
+    private ByteBuffer input;
+    private ByteBuf compressed;
+    private ByteBuf output;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (input != null)
+            FileUtils.clean(input);
+        if (compressed != null && compressed.refCnt() > 0)
+            compressed.release(compressed.refCnt());
+        if (output != null && output.refCnt() > 0)
+            output.release(output.refCnt());
+    }
+
+    @Test
+    public void roundTrip_HappyPath_NotReadabaleByteBuffer() throws IOException
+    {
+        populateInput();
+        compressed = serializer.serialize(compressor, input, VERSION);
+        input.flip();
+        ByteBuffer compressedNioBuffer = compressed.nioBuffer(0, compressed.writerIndex());
+        output = serializer.deserialize(decompressor, new DataInputBuffer(compressedNioBuffer, false), VERSION);
+        validateResults();
+    }
+
+    private void populateInput()
+    {
+        int bufSize = 1 << 14;
+        input = ByteBuffer.allocateDirect(bufSize);
+        for (int i = 0; i < bufSize; i += 4)
+            input.putInt(random.nextInt());
+        input.flip();
+    }
+
+    private void validateResults()
+    {
+        Assert.assertEquals(input.remaining(), output.readableBytes());
+        for (int i = 0; i < input.remaining(); i++)
+            Assert.assertEquals(input.get(i), output.readByte());
+    }
+
+    @Test
+    public void roundTrip_HappyPath_ReadabaleByteBuffer() throws IOException
+    {
+        populateInput();
+        compressed = serializer.serialize(compressor, input, VERSION);
+        input.flip();
+        output = serializer.deserialize(decompressor, new ByteBufRCH(compressed), VERSION);
+        validateResults();
+    }
+
+    private static class ByteBufRCH extends DataInputBuffer implements ReadableByteChannel
+    {
+        public ByteBufRCH(ByteBuf compressed)
+        {
+            super (compressed.nioBuffer(0, compressed.readableBytes()), false);
+        }
+
+        @Override
+        public int read(ByteBuffer dst) throws IOException
+        {
+            int len = dst.remaining();
+            dst.put(buffer);
+            return len;
+        }
+
+        @Override
+        public boolean isOpen()
+        {
+            return true;
+        }
+
+        @Override
+        public void close()
+        {   }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java b/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java
new file mode 100644
index 0000000..a674e6b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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.async;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.UUID;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.big.BigFormat;
+import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.StreamManager;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.async.StreamingInboundHandler.SessionIdentifier;
+import org.apache.cassandra.streaming.messages.CompleteMessage;
+import org.apache.cassandra.streaming.messages.FileMessageHeader;
+import org.apache.cassandra.streaming.messages.IncomingFileMessage;
+import org.apache.cassandra.streaming.messages.StreamInitMessage;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+
+public class StreamingInboundHandlerTest
+{
+    private static final int VERSION = StreamMessage.CURRENT_VERSION;
+    private static final InetSocketAddress REMOTE_ADDR = new InetSocketAddress("127.0.0.2", 0);
+
+    private StreamingInboundHandler handler;
+    private EmbeddedChannel channel;
+    private RebufferingByteBufDataInputPlus buffers;
+    private ByteBuf buf;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Before
+    public void setup()
+    {
+        handler = new StreamingInboundHandler(REMOTE_ADDR, VERSION, null);
+        channel = new EmbeddedChannel(handler);
+        buffers = new RebufferingByteBufDataInputPlus(1 << 9, 1 << 10, channel.config());
+        handler.setPendingBuffers(buffers);
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (buf != null)
+        {
+            while (buf.refCnt() > 0)
+                buf.release();
+        }
+
+        channel.close();
+    }
+
+    @Test
+    public void channelRead_Normal() throws EOFException
+    {
+        Assert.assertEquals(0, buffers.available());
+        int size = 8;
+        buf = channel.alloc().buffer(size);
+        buf.writerIndex(size);
+        channel.writeInbound(buf);
+        Assert.assertEquals(size, buffers.available());
+        Assert.assertFalse(channel.releaseInbound());
+    }
+
+    @Test (expected = EOFException.class)
+    public void channelRead_Closed() throws EOFException
+    {
+        int size = 8;
+        buf = channel.alloc().buffer(size);
+        Assert.assertEquals(1, buf.refCnt());
+        buf.writerIndex(size);
+        handler.close();
+        channel.writeInbound(buf);
+        Assert.assertEquals(0, buffers.available());
+        Assert.assertEquals(0, buf.refCnt());
+        Assert.assertFalse(channel.releaseInbound());
+    }
+
+    @Test
+    public void channelRead_WrongObject() throws EOFException
+    {
+        channel.writeInbound("homer");
+        Assert.assertEquals(0, buffers.available());
+        Assert.assertFalse(channel.releaseInbound());
+    }
+
+    @Test
+    public void StreamDeserializingTask_deriveSession_StreamInitMessage() throws InterruptedException, IOException
+    {
+        StreamInitMessage msg = new StreamInitMessage(REMOTE_ADDR.getAddress(), 0, UUID.randomUUID(), StreamOperation.REPAIR, true, UUID.randomUUID(), PreviewKind.ALL);
+        StreamingInboundHandler.StreamDeserializingTask task = handler.new StreamDeserializingTask(sid -> createSession(sid), null, channel);
+        StreamSession session = task.deriveSession(msg);
+        Assert.assertNotNull(session);
+    }
+
+    private StreamSession createSession(SessionIdentifier sid)
+    {
+        return new StreamSession(sid.from, sid.from, (connectionId, protocolVersion) -> null, sid.sessionIndex, true, UUID.randomUUID(), PreviewKind.ALL);
+    }
+
+    @Test (expected = IllegalStateException.class)
+    public void StreamDeserializingTask_deriveSession_NoSession() throws InterruptedException, IOException
+    {
+        CompleteMessage msg = new CompleteMessage();
+        StreamingInboundHandler.StreamDeserializingTask task = handler.new StreamDeserializingTask(sid -> createSession(sid), null, channel);
+        task.deriveSession(msg);
+    }
+
+    @Test (expected = IllegalStateException.class)
+    public void StreamDeserializingTask_deriveSession_IFM_NoSession() throws InterruptedException, IOException
+    {
+        FileMessageHeader header = new FileMessageHeader(TableId.generate(), REMOTE_ADDR.getAddress(), UUID.randomUUID(), 0, 0,
+                                                         BigFormat.latestVersion, SSTableFormat.Type.BIG, 0, new ArrayList<>(), null, 0, UUID.randomUUID(), 0 , null);
+        IncomingFileMessage msg = new IncomingFileMessage(null, header);
+        StreamingInboundHandler.StreamDeserializingTask task = handler.new StreamDeserializingTask(sid -> StreamManager.instance.findSession(sid.from, sid.planId, sid.sessionIndex), null, channel);
+        task.deriveSession(msg);
+    }
+
+    @Test
+    public void StreamDeserializingTask_deriveSession_IFM_HasSession() throws InterruptedException, IOException
+    {
+        UUID planId = UUID.randomUUID();
+        StreamResultFuture future = StreamResultFuture.initReceivingSide(0, planId, StreamOperation.REPAIR, REMOTE_ADDR.getAddress(), channel, true, UUID.randomUUID(), PreviewKind.ALL);
+        StreamManager.instance.register(future);
+        FileMessageHeader header = new FileMessageHeader(TableId.generate(), REMOTE_ADDR.getAddress(), planId, 0, 0,
+                                                         BigFormat.latestVersion, SSTableFormat.Type.BIG, 0, new ArrayList<>(), null, 0, UUID.randomUUID(), 0 , null);
+        IncomingFileMessage msg = new IncomingFileMessage(null, header);
+        StreamingInboundHandler.StreamDeserializingTask task = handler.new StreamDeserializingTask(sid -> StreamManager.instance.findSession(sid.from, sid.planId, sid.sessionIndex), null, channel);
+        StreamSession session = task.deriveSession(msg);
+        Assert.assertNotNull(session);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java b/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java
index 19e28fd..16b3a76 100644
--- a/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java
+++ b/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java
@@ -28,12 +28,11 @@ import org.apache.cassandra.db.ClusteringComparator;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.io.compress.CompressedSequentialWriter;
 import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.io.util.SequentialWriterOption;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.format.Version;
-import org.apache.cassandra.io.sstable.format.big.BigFormat;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.streaming.compress.CompressedInputStream;
 import org.apache.cassandra.streaming.compress.CompressionInfo;
@@ -174,7 +173,7 @@ public class CompressedInputStreamTest
             testException(sections, info);
             return;
         }
-        CompressedInputStream input = new CompressedInputStream(new ByteArrayInputStream(toRead), info, ChecksumType.CRC32, () -> 1.0);
+        CompressedInputStream input = new CompressedInputStream(new DataInputStreamPlus(new ByteArrayInputStream(toRead)), info, ChecksumType.CRC32, () -> 1.0);
 
         try (DataInputStream in = new DataInputStream(input))
         {
@@ -189,14 +188,14 @@ public class CompressedInputStreamTest
 
     private static void testException(List<Pair<Long, Long>> sections, CompressionInfo info) throws IOException
     {
-        CompressedInputStream input = new CompressedInputStream(new ByteArrayInputStream(new byte[0]), info, ChecksumType.CRC32, () -> 1.0);
+        CompressedInputStream input = new CompressedInputStream(new DataInputStreamPlus(new ByteArrayInputStream(new byte[0])), info, ChecksumType.CRC32, () -> 1.0);
 
         try (DataInputStream in = new DataInputStream(input))
         {
             for (int i = 0; i < sections.size(); i++)
             {
-                input.position(sections.get(i).left);
                 try {
+                    input.position(sections.get(i).left);
                     in.readLong();
                     fail("Should have thrown IOException");
                 }
@@ -208,3 +207,4 @@ public class CompressedInputStreamTest
         }
     }
 }
+


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[10/11] cassandra git commit: switch internode messaging to netty

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/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 41771e7..6caada1 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -17,45 +17,63 @@
  */
 package org.apache.cassandra.net;
 
-import java.io.*;
+import java.io.IOError;
+import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.net.*;
-import java.nio.channels.AsynchronousCloseException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.channels.ServerSocketChannel;
-import java.util.*;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.EnumMap;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Collectors;
-import java.util.stream.StreamSupport;
-
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
-import javax.net.ssl.SSLHandshakeException;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.carrotsearch.hppc.IntObjectMap;
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import io.netty.channel.Channel;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.group.DefaultChannelGroup;
+import org.apache.cassandra.auth.IInternodeAuthenticator;
+import org.apache.cassandra.batchlog.Batch;
 import org.apache.cassandra.concurrent.ExecutorLocals;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.batchlog.Batch;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.CounterMutation;
+import org.apache.cassandra.db.IMutation;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.ReadResponse;
+import org.apache.cassandra.db.SnapshotCommand;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.TruncateResponse;
+import org.apache.cassandra.db.Truncation;
+import org.apache.cassandra.db.WriteResponse;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.BootStrapper;
 import org.apache.cassandra.dht.IPartitioner;
@@ -70,22 +88,32 @@ import org.apache.cassandra.hints.HintResponse;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.ILatencySubscriber;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.metrics.ConnectionMetrics;
 import org.apache.cassandra.metrics.DroppedMessageMetrics;
 import org.apache.cassandra.metrics.MessagingMetrics;
+import org.apache.cassandra.net.async.OutboundMessagingPool;
+import org.apache.cassandra.net.async.NettyFactory;
+import org.apache.cassandra.net.async.NettyFactory.InboundInitializer;
 import org.apache.cassandra.repair.messages.RepairMessage;
 import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.schema.TableId;
-import org.apache.cassandra.security.SSLFactory;
-import org.apache.cassandra.service.*;
+import org.apache.cassandra.service.AbstractWriteResponseHandler;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.service.paxos.PrepareResponse;
 import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.BooleanSerializer;
+import org.apache.cassandra.utils.ExpiringMap;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.NativeLibrary;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.StatusLogger;
+import org.apache.cassandra.utils.UUIDSerializer;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
 public final class MessagingService implements MessagingServiceMBean
@@ -323,7 +351,7 @@ public final class MessagingService implements MessagingServiceMBean
     }};
 
     /**
-     * Messages we receive in IncomingTcpConnection have a Verb that tells us what kind of message it is.
+     * Messages we receive from peers have a Verb that tells us what kind of message it is.
      * Most of the time, this is enough to determine how to deserialize the message payload.
      * The exception is the REQUEST_RESPONSE verb, which just means "a reply to something you told me to do."
      * Traditionally, this was fine since each VerbHandler knew what type of payload it expected, and
@@ -418,12 +446,12 @@ public final class MessagingService implements MessagingServiceMBean
     private final Map<Verb, IVerbHandler> verbHandlers;
 
     @VisibleForTesting
-    final ConcurrentMap<InetAddress, OutboundTcpConnectionPool> connectionManagers = new NonBlockingHashMap<>();
+    public final ConcurrentMap<InetAddress, OutboundMessagingPool> channelManagers = new NonBlockingHashMap<>();
+    final List<ServerChannel> serverChannels = Lists.newArrayList();
 
     private static final Logger logger = LoggerFactory.getLogger(MessagingService.class);
     private static final int LOG_DROPPED_INTERVAL_IN_MS = 5000;
 
-    private final List<SocketThread> socketThreads = Lists.newArrayList();
     private final SimpleCondition listenGate;
 
     /**
@@ -533,9 +561,7 @@ public final class MessagingService implements MessagingServiceMBean
                 maybeAddLatency(expiredCallbackInfo.callback, expiredCallbackInfo.target, pair.right.timeout);
 
                 ConnectionMetrics.totalTimeouts.mark();
-                OutboundTcpConnectionPool cp = getConnectionPool(expiredCallbackInfo.target);
-                if (cp != null)
-                    cp.incrementTimeout();
+                markTimeout(expiredCallbackInfo.target);
 
                 if (expiredCallbackInfo.callback.supportsBackPressure())
                 {
@@ -606,12 +632,9 @@ public final class MessagingService implements MessagingServiceMBean
     {
         if (DatabaseDescriptor.backPressureEnabled() && callback.supportsBackPressure())
         {
-            OutboundTcpConnectionPool cp = getConnectionPool(host);
-            if (cp != null)
-            {
-                BackPressureState backPressureState = cp.getBackPressureState();
+            BackPressureState backPressureState = getBackPressureState(host);
+            if (backPressureState != null)
                 backPressureState.onMessageSent(message);
-            }
         }
     }
 
@@ -626,15 +649,13 @@ public final class MessagingService implements MessagingServiceMBean
     {
         if (DatabaseDescriptor.backPressureEnabled() && callback.supportsBackPressure())
         {
-            OutboundTcpConnectionPool cp = getConnectionPool(host);
-            if (cp != null)
-            {
-                BackPressureState backPressureState = cp.getBackPressureState();
-                if (!timeout)
-                    backPressureState.onResponseReceived();
-                else
-                    backPressureState.onResponseTimeout();
-            }
+            BackPressureState backPressureState = getBackPressureState(host);
+            if (backPressureState == null)
+                return;
+            if (!timeout)
+                backPressureState.onResponseReceived();
+            else
+                backPressureState.onResponseTimeout();
         }
     }
 
@@ -656,14 +677,27 @@ public final class MessagingService implements MessagingServiceMBean
             {
                 if (host.equals(FBUtilities.getBroadcastAddress()))
                     continue;
-                OutboundTcpConnectionPool cp = getConnectionPool(host);
-                if (cp != null)
-                    states.add(cp.getBackPressureState());
+                OutboundMessagingPool pool = getMessagingConnection(host);
+                if (pool != null)
+                    states.add(pool.getBackPressureState());
             }
             backPressure.apply(states, timeoutInNanos, TimeUnit.NANOSECONDS);
         }
     }
 
+    BackPressureState getBackPressureState(InetAddress host)
+    {
+        OutboundMessagingPool messagingConnection = getMessagingConnection(host);
+        return messagingConnection != null ? messagingConnection.getBackPressureState() : null;
+    }
+
+    void markTimeout(InetAddress addr)
+    {
+        OutboundMessagingPool conn = channelManagers.get(addr);
+        if (conn != null)
+            conn.incrementTimeout();
+    }
+
     /**
      * Track latency information for the dynamic snitch
      *
@@ -688,30 +722,25 @@ public final class MessagingService implements MessagingServiceMBean
      */
     public void convict(InetAddress ep)
     {
-        OutboundTcpConnectionPool cp = getConnectionPool(ep);
-        if (cp != null)
-        {
-            logger.trace("Resetting pool for {}", ep);
-            cp.reset();
-        }
-        else
-        {
-            logger.debug("Not resetting pool for {} because internode authenticator said not to connect", ep);
-        }
+        logger.trace("Resetting pool for {}", ep);
+        reset(ep);
     }
 
     public void listen()
     {
         callbacks.reset(); // hack to allow tests to stop/restart MS
         listen(FBUtilities.getLocalAddress());
-        if (DatabaseDescriptor.shouldListenOnBroadcastAddress()
-            && !FBUtilities.getLocalAddress().equals(FBUtilities.getBroadcastAddress()))
-        {
+        if (shouldListenOnBroadcastAddress())
             listen(FBUtilities.getBroadcastAddress());
-        }
         listenGate.signalAll();
     }
 
+    public static boolean shouldListenOnBroadcastAddress()
+    {
+        return DatabaseDescriptor.shouldListenOnBroadcastAddress()
+               && !FBUtilities.getLocalAddress().equals(FBUtilities.getBroadcastAddress());
+    }
+
     /**
      * Listen on the specified port.
      *
@@ -719,80 +748,64 @@ public final class MessagingService implements MessagingServiceMBean
      */
     private void listen(InetAddress localEp) throws ConfigurationException
     {
-        for (ServerSocket ss : getServerSockets(localEp))
+        IInternodeAuthenticator authenticator = DatabaseDescriptor.getInternodeAuthenticator();
+        int receiveBufferSize = DatabaseDescriptor.getInternodeRecvBufferSize();
+
+        if (DatabaseDescriptor.getServerEncryptionOptions().internode_encryption != ServerEncryptionOptions.InternodeEncryption.none)
         {
-            SocketThread th = new SocketThread(ss, "ACCEPT-" + localEp);
-            th.start();
-            socketThreads.add(th);
+            InetSocketAddress localAddr = new InetSocketAddress(localEp, DatabaseDescriptor.getSSLStoragePort());
+            ChannelGroup channelGroup = new DefaultChannelGroup("EncryptedInternodeMessagingGroup", NettyFactory.executorForChannelGroups());
+            InboundInitializer initializer = new InboundInitializer(authenticator, DatabaseDescriptor.getServerEncryptionOptions(), channelGroup);
+            Channel encryptedChannel = NettyFactory.instance.createInboundChannel(localAddr, initializer, receiveBufferSize);
+            serverChannels.add(new ServerChannel(encryptedChannel, channelGroup));
         }
+
+        if (DatabaseDescriptor.getServerEncryptionOptions().internode_encryption != ServerEncryptionOptions.InternodeEncryption.all)
+        {
+            InetSocketAddress localAddr = new InetSocketAddress(localEp, DatabaseDescriptor.getStoragePort());
+            ChannelGroup channelGroup = new DefaultChannelGroup("InternodeMessagingGroup", NettyFactory.executorForChannelGroups());
+            InboundInitializer initializer = new InboundInitializer(authenticator, null, channelGroup);
+            Channel channel = NettyFactory.instance.createInboundChannel(localAddr, initializer, receiveBufferSize);
+            serverChannels.add(new ServerChannel(channel, channelGroup));
+        }
+
+        if (serverChannels.isEmpty())
+            throw new IllegalStateException("no listening channels set up in MessagingService!");
     }
 
-    @SuppressWarnings("resource")
-    private List<ServerSocket> getServerSockets(InetAddress localEp) throws ConfigurationException
+    /**
+     * A simple struct to wrap up the the components needed for each listening socket.
+     */
+    @VisibleForTesting
+    static class ServerChannel
     {
-        final List<ServerSocket> ss = new ArrayList<ServerSocket>(2);
-        if (DatabaseDescriptor.getServerEncryptionOptions().internode_encryption != ServerEncryptionOptions.InternodeEncryption.none)
+        /**
+         * The base {@link Channel} that is doing the spcket listen/accept.
+         */
+        private final Channel channel;
+
+        /**
+         * A group of the open, inbound {@link Channel}s connected to this node. This is mostly interesting so that all of
+         * the inbound connections/channels can be closed when the listening socket itself is being closed.
+         */
+        private final ChannelGroup connectedChannels;
+
+        private ServerChannel(Channel channel, ChannelGroup channelGroup)
         {
-            try
-            {
-                ss.add(SSLFactory.getServerSocket(DatabaseDescriptor.getServerEncryptionOptions(), localEp, DatabaseDescriptor.getSSLStoragePort()));
-            }
-            catch (IOException e)
-            {
-                throw new ConfigurationException("Unable to create ssl socket", e);
-            }
-            // setReuseAddress happens in the factory.
-            logger.info("Starting Encrypted Messaging Service on SSL port {}", DatabaseDescriptor.getSSLStoragePort());
+            this.channel = channel;
+            this.connectedChannels = channelGroup;
         }
 
-        if (DatabaseDescriptor.getServerEncryptionOptions().internode_encryption != ServerEncryptionOptions.InternodeEncryption.all)
+        void close()
         {
-            ServerSocketChannel serverChannel = null;
-            try
-            {
-                serverChannel = ServerSocketChannel.open();
-            }
-            catch (IOException e)
-            {
-                throw new RuntimeException(e);
-            }
-            ServerSocket socket = serverChannel.socket();
-            try
-            {
-                socket.setReuseAddress(true);
-            }
-            catch (SocketException e)
-            {
-                FileUtils.closeQuietly(socket);
-                throw new ConfigurationException("Insufficient permissions to setReuseAddress", e);
-            }
-            InetSocketAddress address = new InetSocketAddress(localEp, DatabaseDescriptor.getStoragePort());
-            try
-            {
-                socket.bind(address,500);
-            }
-            catch (BindException e)
-            {
-                FileUtils.closeQuietly(socket);
-                if (e.getMessage().contains("in use"))
-                    throw new ConfigurationException(address + " is in use by another process.  Change listen_address:storage_port in cassandra.yaml to values that do not conflict with other services");
-                else if (e.getMessage().contains("Cannot assign requested address"))
-                    throw new ConfigurationException("Unable to bind to address " + address
-                                                     + ". Set listen_address in cassandra.yaml to an interface you can bind to, e.g., your private IP address on EC2");
-                else
-                    throw new RuntimeException(e);
-            }
-            catch (IOException e)
-            {
-                FileUtils.closeQuietly(socket);
-                throw new RuntimeException(e);
-            }
-            String nic = FBUtilities.getNetworkInterface(localEp);
-            logger.info("Starting Messaging Service on {}:{}{}", localEp, DatabaseDescriptor.getStoragePort(),
-                        nic == null? "" : String.format(" (%s)", nic));
-            ss.add(socket);
+            channel.close().syncUninterruptibly();
+            connectedChannels.close().syncUninterruptibly();
+        }
+        int size()
+
+        {
+            return connectedChannels.size();
         }
-        return ss;
     }
 
     public void waitUntilListening()
@@ -812,53 +825,42 @@ public final class MessagingService implements MessagingServiceMBean
         return listenGate.isSignaled();
     }
 
+
     public void destroyConnectionPool(InetAddress to)
     {
-        OutboundTcpConnectionPool cp = connectionManagers.get(to);
-        if (cp == null)
-            return;
-        cp.close();
-        connectionManagers.remove(to);
+        OutboundMessagingPool pool = channelManagers.remove(to);
+        if (pool != null)
+            pool.close(true);
     }
 
     /**
-     * Get a connection pool to the specified endpoint. Constructs one if none exists.
+     * Reconnect to the peer using the given {@code addr}. Outstanding messages in each channel will be sent on the
+     * current channel. Typically this function is used for something like EC2 public IP addresses which need to be used
+     * for communication between EC2 regions.
      *
-     * Can return null if the InternodeAuthenticator fails to authenticate the node.
-     * @param to
-     * @return The connection pool or null if internode authenticator says not to
+     * @param address IP Address to identify the peer
+     * @param preferredAddress IP Address to use (and prefer) going forward for connecting to the peer
      */
-    public OutboundTcpConnectionPool getConnectionPool(InetAddress to)
+    public void reconnectWithNewIp(InetAddress address, InetAddress preferredAddress)
     {
-        OutboundTcpConnectionPool cp = connectionManagers.get(to);
-        if (cp == null)
-        {
-            //Don't attempt to connect to nodes that won't (or shouldn't) authenticate anyways
-            if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(to, OutboundTcpConnectionPool.portFor(to)))
-                return null;
+        SystemKeyspace.updatePreferredIP(address, preferredAddress);
 
-            cp = new OutboundTcpConnectionPool(to, backPressure.newState(to));
-            OutboundTcpConnectionPool existingPool = connectionManagers.putIfAbsent(to, cp);
-            if (existingPool != null)
-                cp = existingPool;
-            else
-                cp.start();
-        }
-        cp.waitForStarted();
-        return cp;
+        OutboundMessagingPool messagingPool = channelManagers.get(address);
+        if (messagingPool != null)
+            messagingPool.reconnectWithNewIp(new InetSocketAddress(preferredAddress, portFor(address)));
     }
 
-    /**
-     * Get a connection for a message to a specific endpoint. Constructs one if none exists.
-     *
-     * Can return null if the InternodeAuthenticator fails to authenticate the node.
-     * @param to
-     * @return The connection or null if internode authenticator says not to
-     */
-    public OutboundTcpConnection getConnection(InetAddress to, MessageOut msg)
+    private void reset(InetAddress address)
     {
-        OutboundTcpConnectionPool cp = getConnectionPool(to);
-        return cp == null ? null : cp.getConnection(msg);
+        OutboundMessagingPool messagingPool = channelManagers.remove(address);
+        if (messagingPool != null)
+            messagingPool.close(false);
+    }
+
+    public InetAddress getCurrentEndpoint(InetAddress publicAddress)
+    {
+        OutboundMessagingPool messagingPool = getMessagingConnection(publicAddress);
+        return messagingPool != null ? messagingPool.getPreferredRemoteAddr().getAddress() : null;
     }
 
     /**
@@ -1008,12 +1010,9 @@ public final class MessagingService implements MessagingServiceMBean
             if (!ms.allowOutgoingMessage(message, id, to))
                 return;
 
-        // get pooled connection (really, connection queue)
-        OutboundTcpConnection connection = getConnection(to, message);
-
-        // write it
-        if (connection != null)
-            connection.enqueue(message, id);
+        OutboundMessagingPool outboundMessagingPool = getMessagingConnection(to);
+        if (outboundMessagingPool != null)
+            outboundMessagingPool.sendMessage(message, id);
     }
 
     public <T> AsyncOneResponse<T> sendRR(MessageOut message, InetAddress to)
@@ -1049,18 +1048,17 @@ public final class MessagingService implements MessagingServiceMBean
         // attempt to humor tests that try to stop and restart MS
         try
         {
-            for (SocketThread th : socketThreads)
-                try
-                {
-                    th.close();
-                }
-                catch (IOException e)
-                {
-                    // see https://issues.apache.org/jira/browse/CASSANDRA-10545
-                    handleIOExceptionOnClose(e);
-                }
+            // first close the recieve channels
+            for (ServerChannel serverChannel : serverChannels)
+                serverChannel.close();
+
+            // now close the send channels
+            for (OutboundMessagingPool pool : channelManagers.values())
+                pool.close(false);
+
+            NettyFactory.instance.close();
         }
-        catch (IOException e)
+        catch (Exception e)
         {
             throw new IOError(e);
         }
@@ -1281,109 +1279,13 @@ public final class MessagingService implements MessagingServiceMBean
         return ret;
     }
 
-    @VisibleForTesting
-    public static class SocketThread extends Thread
-    {
-        private final ServerSocket server;
-        @VisibleForTesting
-        public final Set<Closeable> connections = Sets.newConcurrentHashSet();
-
-        SocketThread(ServerSocket server, String name)
-        {
-            super(name);
-            this.server = server;
-        }
-
-        @SuppressWarnings("resource")
-        public void run()
-        {
-            while (!server.isClosed())
-            {
-                Socket socket = null;
-                try
-                {
-                    socket = server.accept();
-                    if (!authenticate(socket))
-                    {
-                        logger.trace("remote failed to authenticate");
-                        socket.close();
-                        continue;
-                    }
-
-                    socket.setKeepAlive(true);
-                    socket.setSoTimeout(2 * OutboundTcpConnection.WAIT_FOR_VERSION_MAX_TIME);
-                    // 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.trace("Connection version {} from {}", version, socket.getInetAddress());
-                    socket.setSoTimeout(0);
-
-                    Thread thread = isStream
-                                  ? new IncomingStreamingConnection(version, socket, connections)
-                                  : new IncomingTcpConnection(version, MessagingService.getBits(header, 2, 1) == 1, socket, connections);
-                    thread.start();
-                    connections.add((Closeable) thread);
-                }
-                catch (AsynchronousCloseException e)
-                {
-                    // this happens when another thread calls close().
-                    logger.trace("Asynchronous close seen by server thread");
-                    break;
-                }
-                catch (ClosedChannelException e)
-                {
-                    logger.trace("MessagingService server thread already closed");
-                    break;
-                }
-                catch (SSLHandshakeException e)
-                {
-                    logger.error("SSL handshake error for inbound connection from " + socket, e);
-                    FileUtils.closeQuietly(socket);
-                }
-                catch (Throwable t)
-                {
-                    logger.trace("Error reading the socket {}", socket, t);
-                    FileUtils.closeQuietly(socket);
-                }
-            }
-            logger.info("MessagingService has terminated the accept() thread");
-        }
-
-        void close() throws IOException
-        {
-            logger.trace("Closing accept() thread");
-
-            try
-            {
-                server.close();
-            }
-            catch (IOException e)
-            {
-                // see https://issues.apache.org/jira/browse/CASSANDRA-8220
-                // see https://issues.apache.org/jira/browse/CASSANDRA-12513
-                handleIOExceptionOnClose(e);
-            }
-            for (Closeable connection : connections)
-            {
-                connection.close();
-            }
-        }
-
-        private boolean authenticate(Socket socket)
-        {
-            return DatabaseDescriptor.getInternodeAuthenticator().authenticate(socket.getInetAddress(), socket.getPort());
-        }
-    }
 
     private static void handleIOExceptionOnClose(IOException e) throws IOException
     {
         // dirty hack for clean shutdown on OSX w/ Java >= 1.8.0_20
         // see https://bugs.openjdk.java.net/browse/JDK-8050499;
         // also CASSANDRA-12513
-        if ("Mac OS X".equals(System.getProperty("os.name")))
+        if (NativeLibrary.osType == NativeLibrary.OSType.MAC)
         {
             switch (e.getMessage())
             {
@@ -1398,79 +1300,73 @@ public final class MessagingService implements MessagingServiceMBean
 
     public Map<String, Integer> getLargeMessagePendingTasks()
     {
-        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessages.getPendingMessages());
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessageChannel.getPendingMessages());
         return pendingTasks;
     }
 
-    public int getLargeMessagePendingTasks(InetAddress address)
-    {
-        OutboundTcpConnectionPool connection = connectionManagers.get(address);
-        return connection == null ? 0 : connection.largeMessages.getPendingMessages();
-    }
-
     public Map<String, Long> getLargeMessageCompletedTasks()
     {
-        Map<String, Long> completedTasks = new HashMap<String, Long>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessages.getCompletedMesssages());
+        Map<String, Long> completedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessageChannel.getCompletedMessages());
         return completedTasks;
     }
 
     public Map<String, Long> getLargeMessageDroppedTasks()
     {
-        Map<String, Long> droppedTasks = new HashMap<String, Long>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessages.getDroppedMessages());
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessageChannel.getDroppedMessages());
         return droppedTasks;
     }
 
     public Map<String, Integer> getSmallMessagePendingTasks()
     {
-        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessages.getPendingMessages());
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessageChannel.getPendingMessages());
         return pendingTasks;
     }
 
     public Map<String, Long> getSmallMessageCompletedTasks()
     {
-        Map<String, Long> completedTasks = new HashMap<String, Long>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessages.getCompletedMesssages());
+        Map<String, Long> completedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessageChannel.getCompletedMessages());
         return completedTasks;
     }
 
     public Map<String, Long> getSmallMessageDroppedTasks()
     {
-        Map<String, Long> droppedTasks = new HashMap<String, Long>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessages.getDroppedMessages());
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessageChannel.getDroppedMessages());
         return droppedTasks;
     }
 
     public Map<String, Integer> getGossipMessagePendingTasks()
     {
-        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipMessages.getPendingMessages());
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipChannel.getPendingMessages());
         return pendingTasks;
     }
 
     public Map<String, Long> getGossipMessageCompletedTasks()
     {
-        Map<String, Long> completedTasks = new HashMap<String, Long>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipMessages.getCompletedMesssages());
+        Map<String, Long> completedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipChannel.getCompletedMessages());
         return completedTasks;
     }
 
     public Map<String, Long> getGossipMessageDroppedTasks()
     {
-        Map<String, Long> droppedTasks = new HashMap<String, Long>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipMessages.getDroppedMessages());
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipChannel.getDroppedMessages());
         return droppedTasks;
     }
 
@@ -1490,8 +1386,8 @@ public final class MessagingService implements MessagingServiceMBean
 
     public Map<String, Long> getTimeoutsPerHost()
     {
-        Map<String, Long> result = new HashMap<String, Long>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry: connectionManagers.entrySet())
+        Map<String, Long> result = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
         {
             String ip = entry.getKey().getHostAddress();
             long recent = entry.getValue().getTimeouts();
@@ -1502,8 +1398,8 @@ public final class MessagingService implements MessagingServiceMBean
 
     public Map<String, Double> getBackPressurePerHost()
     {
-        Map<String, Double> map = new HashMap<>(connectionManagers.size());
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
+        Map<String, Double> map = new HashMap<>(channelManagers.size());
+        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
             map.put(entry.getKey().getHostAddress(), entry.getValue().getBackPressureState().getBackPressureRateLimit());
 
         return map;
@@ -1540,9 +1436,72 @@ public final class MessagingService implements MessagingServiceMBean
                                                    bounds.left.getPartitioner().getClass().getName()));
     }
 
+    private OutboundMessagingPool getMessagingConnection(InetAddress to)
+    {
+        OutboundMessagingPool pool = channelManagers.get(to);
+        if (pool == null)
+        {
+            final boolean secure = isEncryptedConnection(to);
+            final int port = portFor(secure);
+            if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(to, port))
+                return null;
+
+            InetSocketAddress preferredRemote = new InetSocketAddress(SystemKeyspace.getPreferredIP(to), port);
+            InetSocketAddress local = new InetSocketAddress(FBUtilities.getLocalAddress(), 0);
+            ServerEncryptionOptions encryptionOptions = secure ? DatabaseDescriptor.getServerEncryptionOptions() : null;
+            IInternodeAuthenticator authenticator = DatabaseDescriptor.getInternodeAuthenticator();
+
+            pool = new OutboundMessagingPool(preferredRemote, local, encryptionOptions, backPressure.newState(to), authenticator);
+            OutboundMessagingPool existing = channelManagers.putIfAbsent(to, pool);
+            if (existing != null)
+            {
+                pool.close(false);
+                pool = existing;
+            }
+        }
+        return pool;
+    }
+
+    public static int portFor(InetAddress addr)
+    {
+        final boolean secure = isEncryptedConnection(addr);
+        return portFor(secure);
+    }
+
+    private static int portFor(boolean secure)
+    {
+        return secure ? DatabaseDescriptor.getSSLStoragePort() : DatabaseDescriptor.getStoragePort();
+    }
+
     @VisibleForTesting
-    public List<SocketThread> getSocketThreads()
+    boolean isConnected(InetAddress address, MessageOut messageOut)
     {
-        return socketThreads;
+        OutboundMessagingPool pool = channelManagers.get(address);
+        if (pool == null)
+            return false;
+        return pool.getConnection(messageOut).isConnected();
+    }
+
+    public static boolean isEncryptedConnection(InetAddress address)
+    {
+        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+        switch (DatabaseDescriptor.getServerEncryptionOptions().internode_encryption)
+        {
+            case none:
+                return false; // if nothing needs to be encrypted then return immediately.
+            case all:
+                break;
+            case dc:
+                if (snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddress())))
+                    return false;
+                break;
+            case rack:
+                // for rack then check if the DC's are the same.
+                if (snitch.getRack(address).equals(snitch.getRack(FBUtilities.getBroadcastAddress()))
+                    && snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddress())))
+                    return false;
+                break;
+        }
+        return true;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
deleted file mode 100644
index 42abbe6..0000000
--- a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
+++ /dev/null
@@ -1,693 +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.net;
-
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.net.SocketException;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.*;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.zip.Checksum;
-
-import javax.net.ssl.SSLHandshakeException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.netty.util.concurrent.FastThreadLocalThread;
-import net.jpountz.lz4.LZ4BlockOutputStream;
-import net.jpountz.lz4.LZ4Compressor;
-import net.jpountz.lz4.LZ4Factory;
-import net.jpountz.xxhash.XXHashFactory;
-
-import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
-import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.CoalescingStrategies;
-import org.apache.cassandra.utils.CoalescingStrategies.Coalescable;
-import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis;
-import org.apache.cassandra.utils.UUIDGen;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.Uninterruptibles;
-
-public class OutboundTcpConnection extends FastThreadLocalThread
-{
-    private static final Logger logger = LoggerFactory.getLogger(OutboundTcpConnection.class);
-
-    private static final String PREFIX = Config.PROPERTY_PREFIX;
-
-    /*
-     * Enabled/disable TCP_NODELAY for intradc connections. Defaults to enabled.
-     */
-    private static final String INTRADC_TCP_NODELAY_PROPERTY = PREFIX + "otc_intradc_tcp_nodelay";
-    private static final boolean INTRADC_TCP_NODELAY = Boolean.parseBoolean(System.getProperty(INTRADC_TCP_NODELAY_PROPERTY, "true"));
-
-    /*
-     * Size of buffer in output stream
-     */
-    private static final String BUFFER_SIZE_PROPERTY = PREFIX + "otc_buffer_size";
-    private static final int BUFFER_SIZE = Integer.getInteger(BUFFER_SIZE_PROPERTY, 1024 * 64);
-
-    //Size of 3 elements added to every message
-    private static final int PROTOCOL_MAGIC_ID_TIMESTAMP_SIZE = 12;
-
-    public static final int MAX_COALESCED_MESSAGES = 128;
-
-    private static CoalescingStrategy newCoalescingStrategy(String displayName)
-    {
-        return CoalescingStrategies.newCoalescingStrategy(DatabaseDescriptor.getOtcCoalescingStrategy(),
-                                                          DatabaseDescriptor.getOtcCoalescingWindow(),
-                                                          logger,
-                                                          displayName);
-    }
-
-    static
-    {
-        String strategy = DatabaseDescriptor.getOtcCoalescingStrategy();
-        switch (strategy)
-        {
-        case "TIMEHORIZON":
-            break;
-        case "MOVINGAVERAGE":
-        case "FIXED":
-        case "DISABLED":
-            logger.info("OutboundTcpConnection using coalescing strategy {}", strategy);
-            break;
-            default:
-                //Check that it can be loaded
-                newCoalescingStrategy("dummy");
-        }
-
-        int coalescingWindow = DatabaseDescriptor.getOtcCoalescingWindow();
-        if (coalescingWindow != Config.otc_coalescing_window_us_default)
-            logger.info("OutboundTcpConnection coalescing window set to {}μs", coalescingWindow);
-
-        if (coalescingWindow < 0)
-            throw new ExceptionInInitializerError(
-                    "Value provided for coalescing window must be greater than 0: " + coalescingWindow);
-
-        int otc_backlog_expiration_interval_in_ms = DatabaseDescriptor.getOtcBacklogExpirationInterval();
-        if (otc_backlog_expiration_interval_in_ms != Config.otc_backlog_expiration_interval_ms_default)
-            logger.info("OutboundTcpConnection backlog expiration interval set to to {}ms", otc_backlog_expiration_interval_in_ms);
-    }
-
-    private static final MessageOut<?> CLOSE_SENTINEL = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE);
-    private volatile boolean isStopped = false;
-
-    private static final int OPEN_RETRY_DELAY = 100; // ms between retries
-    public static final int WAIT_FOR_VERSION_MAX_TIME = 5000;
-    private static final int NO_VERSION = Integer.MIN_VALUE;
-
-    static final int LZ4_HASH_SEED = 0x9747b28c;
-
-    private final BlockingQueue<QueuedMessage> backlog = new LinkedBlockingQueue<>();
-    private static final String BACKLOG_PURGE_SIZE_PROPERTY = PREFIX + "otc_backlog_purge_size";
-    @VisibleForTesting
-    static final int BACKLOG_PURGE_SIZE = Integer.getInteger(BACKLOG_PURGE_SIZE_PROPERTY, 1024);
-    private final AtomicBoolean backlogExpirationActive = new AtomicBoolean(false);
-    private volatile long backlogNextExpirationTime;
-
-    private final OutboundTcpConnectionPool poolReference;
-
-    private final CoalescingStrategy cs;
-    private DataOutputStreamPlus out;
-    private Socket socket;
-    private volatile long completed;
-    private final AtomicLong dropped = new AtomicLong();
-    private volatile int currentMsgBufferCount = 0;
-    private volatile int targetVersion;
-
-    public OutboundTcpConnection(OutboundTcpConnectionPool pool, String name)
-    {
-        super("MessagingService-Outgoing-" + pool.endPoint() + "-" + name);
-        this.poolReference = pool;
-        cs = newCoalescingStrategy(pool.endPoint().getHostAddress());
-
-        // We want to use the most precise version we know because while there is version detection on connect(),
-        // the target version might be accessed by the pool (in getConnection()) before we actually connect (as we
-        // connect when the first message is submitted). Note however that the only case where we'll connect
-        // without knowing the true version of a node is if that node is a seed (otherwise, we can't know a node
-        // unless it has been gossiped to us or it has connected to us and in both case this sets the version) and
-        // in that case we won't rely on that targetVersion before we're actually connected and so the version
-        // detection in connect() will do its job.
-        targetVersion = MessagingService.instance().getVersion(pool.endPoint());
-    }
-
-    private static boolean isLocalDC(InetAddress targetHost)
-    {
-        String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(targetHost);
-        String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
-        return remoteDC.equals(localDC);
-    }
-
-    public void enqueue(MessageOut<?> message, int id)
-    {
-        long nanoTime = System.nanoTime();
-        expireMessages(nanoTime);
-        try
-        {
-            backlog.put(new QueuedMessage(message, id, nanoTime));
-        }
-        catch (InterruptedException e)
-        {
-            throw new AssertionError(e);
-        }
-    }
-
-    /**
-     * This is a helper method for unit testing. Disclaimer: Do not use this method outside unit tests, as
-     * this method is iterating the queue which can be an expensive operation (CPU time, queue locking).
-     *
-     * @return true, if the queue contains at least one expired element
-     */
-    @VisibleForTesting // (otherwise = VisibleForTesting.NONE)
-    boolean backlogContainsExpiredMessages(long nowNanos)
-    {
-        return backlog.stream().anyMatch(entry -> entry.isTimedOut(nowNanos));
-    }
-
-    void closeSocket(boolean destroyThread)
-    {
-        logger.debug("Enqueuing socket close for {}", poolReference.endPoint());
-        isStopped = destroyThread; // Exit loop to stop the thread
-        backlog.clear();
-        // in the "destroyThread = true" case, enqueuing the sentinel is important mostly to unblock the backlog.take()
-        // (via the CoalescingStrategy) in case there's a data race between this method enqueuing the sentinel
-        // and run() clearing the backlog on connection failure.
-        enqueue(CLOSE_SENTINEL, -1);
-    }
-
-    void softCloseSocket()
-    {
-        enqueue(CLOSE_SENTINEL, -1);
-    }
-
-    public int getTargetVersion()
-    {
-        return targetVersion;
-    }
-
-    public void run()
-    {
-        final int drainedMessageSize = MAX_COALESCED_MESSAGES;
-        // keeping list (batch) size small for now; that way we don't have an unbounded array (that we never resize)
-        final List<QueuedMessage> drainedMessages = new ArrayList<>(drainedMessageSize);
-
-        outer:
-        while (!isStopped)
-        {
-            try
-            {
-                cs.coalesce(backlog, drainedMessages, drainedMessageSize);
-            }
-            catch (InterruptedException e)
-            {
-                throw new AssertionError(e);
-            }
-
-            int count = currentMsgBufferCount = drainedMessages.size();
-
-            //The timestamp of the first message has already been provided to the coalescing strategy
-            //so skip logging it.
-            inner:
-            for (QueuedMessage qm : drainedMessages)
-            {
-                try
-                {
-                    MessageOut<?> m = qm.message;
-                    if (m == CLOSE_SENTINEL)
-                    {
-                        disconnect();
-                        if (isStopped)
-                            break outer;
-                        continue;
-                    }
-
-                    if (qm.isTimedOut(System.nanoTime()))
-                        dropped.incrementAndGet();
-                    else if (socket != null || connect())
-                        writeConnected(qm, count == 1 && backlog.isEmpty());
-                    else
-                    {
-                        // Not connected! Clear out the queue, else gossip messages back up. Update dropped
-                        // statistics accordingly. Hint: The statistics may be slightly too low, if messages
-                        // are added between the calls of backlog.size() and backlog.clear()
-                        dropped.addAndGet(backlog.size());
-                        backlog.clear();
-                        break inner;
-                    }
-                }
-                catch (InternodeAuthFailed e)
-                {
-                    logger.warn("Internode auth failed connecting to {}", poolReference.endPoint());
-                    //Remove the connection pool and other thread so messages aren't queued
-                    MessagingService.instance().destroyConnectionPool(poolReference.endPoint());
-                }
-                catch (Exception e)
-                {
-                    JVMStabilityInspector.inspectThrowable(e);
-                    // really shouldn't get here, as exception handling in writeConnected() is reasonably robust
-                    // but we want to catch anything bad we don't drop the messages in the current batch
-                    logger.error("error processing a message intended for {}", poolReference.endPoint(), e);
-                }
-                currentMsgBufferCount = --count;
-            }
-            // Update dropped statistics by the number of unprocessed drainedMessages
-            dropped.addAndGet(currentMsgBufferCount);
-            drainedMessages.clear();
-        }
-    }
-
-    public int getPendingMessages()
-    {
-        return backlog.size() + currentMsgBufferCount;
-    }
-
-    public long getCompletedMesssages()
-    {
-        return completed;
-    }
-
-    public long getDroppedMessages()
-    {
-        return dropped.get();
-    }
-
-    private boolean shouldCompressConnection()
-    {
-        // assumes version >= 1.2
-        return DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.all
-               || (DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.dc && !isLocalDC(poolReference.endPoint()));
-    }
-
-    private void writeConnected(QueuedMessage qm, boolean flush)
-    {
-        try
-        {
-            byte[] sessionBytes = qm.message.parameters.get(Tracing.TRACE_HEADER);
-            if (sessionBytes != null)
-            {
-                UUID sessionId = UUIDGen.getUUID(ByteBuffer.wrap(sessionBytes));
-                TraceState state = Tracing.instance.get(sessionId);
-                String message = String.format("Sending %s message to %s message size %d bytes", qm.message.verb,
-                                               poolReference.endPoint(),
-                                               qm.message.serializedSize(targetVersion) + PROTOCOL_MAGIC_ID_TIMESTAMP_SIZE);
-                // session may have already finished; see CASSANDRA-5668
-                if (state == null)
-                {
-                    byte[] traceTypeBytes = qm.message.parameters.get(Tracing.TRACE_TYPE);
-                    Tracing.TraceType traceType = traceTypeBytes == null ? Tracing.TraceType.QUERY : Tracing.TraceType.deserialize(traceTypeBytes[0]);
-                    Tracing.instance.trace(ByteBuffer.wrap(sessionBytes), message, traceType.getTTL());
-                }
-                else
-                {
-                    state.trace(message);
-                    if (qm.message.verb == MessagingService.Verb.REQUEST_RESPONSE)
-                        Tracing.instance.doneWithNonLocalSession(state);
-                }
-            }
-
-            long timestampMillis = NanoTimeToCurrentTimeMillis.convert(qm.timestampNanos);
-            writeInternal(qm.message, qm.id, timestampMillis);
-
-            completed++;
-            if (flush)
-                out.flush();
-        }
-        catch (Throwable e)
-        {
-            JVMStabilityInspector.inspectThrowable(e);
-            disconnect();
-            if (e instanceof IOException || e.getCause() instanceof IOException)
-            {
-                logger.debug("Error writing to {}", poolReference.endPoint(), e);
-
-                // If we haven't retried this message yet, put it back on the queue to retry after re-connecting.
-                // See CASSANDRA-5393 and CASSANDRA-12192.
-                if (qm.shouldRetry())
-                {
-                    try
-                    {
-                        backlog.put(new RetriedQueuedMessage(qm));
-                    }
-                    catch (InterruptedException e1)
-                    {
-                        throw new AssertionError(e1);
-                    }
-                }
-            }
-            else
-            {
-                // Non IO exceptions are likely a programming error so let's not silence them
-                logger.error("error writing to {}", poolReference.endPoint(), e);
-            }
-        }
-    }
-
-    private void writeInternal(MessageOut<?> message, int id, long timestamp) throws IOException
-    {
-        //If you add/remove fields before the message don't forget to update PROTOCOL_MAGIC_ID_TIMESTAMP_SIZE
-        out.writeInt(MessagingService.PROTOCOL_MAGIC);
-        out.writeInt(id);
-
-        // int cast cuts off the high-order half of the timestamp, which we can assume remains
-        // the same between now and when the recipient reconstructs it.
-        out.writeInt((int) timestamp);
-        message.serialize(out, targetVersion);
-    }
-
-    private static void writeHeader(DataOutput out, int version, boolean compressionEnabled) throws IOException
-    {
-        // 2 bits: unused.  used to be "serializer type," which was always Binary
-        // 1 bit: compression
-        // 1 bit: streaming mode
-        // 3 bits: unused
-        // 8 bits: version
-        // 15 bits: unused
-        int header = 0;
-        if (compressionEnabled)
-            header |= 4;
-        header |= (version << 8);
-        out.writeInt(header);
-    }
-
-    private void disconnect()
-    {
-        if (socket != null)
-        {
-            try
-            {
-                socket.close();
-                logger.debug("Socket to {} closed", poolReference.endPoint());
-            }
-            catch (IOException e)
-            {
-                logger.debug("Exception closing connection to {}", poolReference.endPoint(), e);
-            }
-            out = null;
-            socket = null;
-        }
-    }
-
-    @SuppressWarnings("resource")
-    private boolean connect() throws InternodeAuthFailed
-    {
-        InetAddress endpoint = poolReference.endPoint();
-        if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(endpoint, poolReference.portFor(endpoint)))
-        {
-            throw new InternodeAuthFailed();
-        }
-
-        logger.debug("Attempting to connect to {}", endpoint);
-
-
-        long start = System.nanoTime();
-        long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getRpcTimeout());
-        while (System.nanoTime() - start < timeout)
-        {
-            targetVersion = MessagingService.instance().getVersion(endpoint);
-            try
-            {
-                socket = poolReference.newSocket();
-                socket.setKeepAlive(true);
-                if (isLocalDC(endpoint))
-                {
-                    socket.setTcpNoDelay(INTRADC_TCP_NODELAY);
-                }
-                else
-                {
-                    socket.setTcpNoDelay(DatabaseDescriptor.getInterDCTcpNoDelay());
-                }
-                if (DatabaseDescriptor.getInternodeSendBufferSize() > 0)
-                {
-                    try
-                    {
-                        socket.setSendBufferSize(DatabaseDescriptor.getInternodeSendBufferSize());
-                    }
-                    catch (SocketException se)
-                    {
-                        logger.warn("Failed to set send buffer size on internode socket.", se);
-                    }
-                }
-
-                // SocketChannel may be null when using SSL
-                WritableByteChannel ch = socket.getChannel();
-                out = new BufferedDataOutputStreamPlus(ch != null ? ch : Channels.newChannel(socket.getOutputStream()), BUFFER_SIZE);
-
-                out.writeInt(MessagingService.PROTOCOL_MAGIC);
-                writeHeader(out, targetVersion, shouldCompressConnection());
-                out.flush();
-
-                DataInputStream in = new DataInputStream(socket.getInputStream());
-                int maxTargetVersion = handshakeVersion(in);
-                if (maxTargetVersion == NO_VERSION)
-                {
-                    // no version is returned, so disconnect an try again
-                    logger.trace("Target max version is {}; no version information yet, will retry", maxTargetVersion);
-                    disconnect();
-                    continue;
-                }
-                else
-                {
-                    MessagingService.instance().setVersion(endpoint, maxTargetVersion);
-                }
-
-                if (targetVersion > maxTargetVersion)
-                {
-                    logger.trace("Target max version is {}; will reconnect with that version", maxTargetVersion);
-                    try
-                    {
-                        if (DatabaseDescriptor.getSeeds().contains(endpoint))
-                            logger.warn("Seed gossip version is {}; will not connect with that version", maxTargetVersion);
-                    }
-                    catch (Throwable e)
-                    {
-                        // If invalid yaml has been added to the config since startup, getSeeds() will throw an AssertionError
-                        // Additionally, third party seed providers may throw exceptions if network is flakey
-                        // Regardless of what's thrown, we must catch it, disconnect, and try again
-                        JVMStabilityInspector.inspectThrowable(e);
-                        logger.warn("Configuration error prevented outbound connection: {}", e.getLocalizedMessage());
-                    }
-                    finally
-                    {
-                        disconnect();
-                        return false;
-                    }
-                }
-
-                if (targetVersion < maxTargetVersion && targetVersion < MessagingService.current_version)
-                {
-                    logger.trace("Detected higher max version {} (using {}); will reconnect when queued messages are done",
-                                 maxTargetVersion, targetVersion);
-                    softCloseSocket();
-                }
-
-                out.writeInt(MessagingService.current_version);
-                CompactEndpointSerializationHelper.serialize(FBUtilities.getBroadcastAddress(), out);
-                if (shouldCompressConnection())
-                {
-                    out.flush();
-                    logger.trace("Upgrading OutputStream to {} to be compressed", endpoint);
-
-                    // TODO: custom LZ4 OS that supports BB write methods
-                    LZ4Compressor compressor = LZ4Factory.fastestInstance().fastCompressor();
-                    Checksum checksum = XXHashFactory.fastestInstance().newStreamingHash32(LZ4_HASH_SEED).asChecksum();
-                    out = new WrappedDataOutputStreamPlus(new LZ4BlockOutputStream(socket.getOutputStream(),
-                                                                        1 << 14,  // 16k block size
-                                                                        compressor,
-                                                                        checksum,
-                                                                        true)); // no async flushing
-                }
-                logger.debug("Done connecting to {}", endpoint);
-                return true;
-            }
-            catch (SSLHandshakeException e)
-            {
-                logger.error("SSL handshake error for outbound connection to " + socket, e);
-                socket = null;
-                // SSL errors won't be recoverable within timeout period so we'll just abort
-                return false;
-            }
-            catch (IOException e)
-            {
-                socket = null;
-                logger.debug("Unable to connect to {}", endpoint, e);
-                Uninterruptibles.sleepUninterruptibly(OPEN_RETRY_DELAY, TimeUnit.MILLISECONDS);
-            }
-        }
-        return false;
-    }
-
-    private int handshakeVersion(final DataInputStream inputStream)
-    {
-        final AtomicInteger version = new AtomicInteger(NO_VERSION);
-        final CountDownLatch versionLatch = new CountDownLatch(1);
-        NamedThreadFactory.createThread(() ->
-        {
-            try
-            {
-                logger.info("Handshaking version with {}", poolReference.endPoint());
-                version.set(inputStream.readInt());
-            }
-            catch (IOException ex)
-            {
-                final String msg = "Cannot handshake version with " + poolReference.endPoint();
-                if (logger.isTraceEnabled())
-                    logger.trace(msg, ex);
-                else
-                    logger.info(msg);
-            }
-            finally
-            {
-                //unblock the waiting thread on either success or fail
-                versionLatch.countDown();
-            }
-        }, "HANDSHAKE-" + poolReference.endPoint()).start();
-
-        try
-        {
-            versionLatch.await(WAIT_FOR_VERSION_MAX_TIME, TimeUnit.MILLISECONDS);
-        }
-        catch (InterruptedException ex)
-        {
-            throw new AssertionError(ex);
-        }
-        return version.get();
-    }
-
-    /**
-     * Expire elements from the queue if the queue is pretty full and expiration is not already in progress.
-     * This method will only remove droppable expired entries. If no such element exists, nothing is removed from the queue.
-     *
-     * @param timestampNanos The current time as from System.nanoTime()
-     */
-    @VisibleForTesting
-    void expireMessages(long timestampNanos)
-    {
-        if (backlog.size() <= BACKLOG_PURGE_SIZE)
-            return; // Plenty of space
-
-        if (backlogNextExpirationTime - timestampNanos > 0)
-            return; // Expiration is not due.
-
-        /**
-         * Expiration is an expensive process. Iterating the queue locks the queue for both writes and
-         * reads during iter.next() and iter.remove(). Thus letting only a single Thread do expiration.
-         */
-        if (backlogExpirationActive.compareAndSet(false, true))
-        {
-            try
-            {
-                Iterator<QueuedMessage> iter = backlog.iterator();
-                while (iter.hasNext())
-                {
-                    QueuedMessage qm = iter.next();
-                    if (!qm.droppable)
-                        continue;
-                    if (!qm.isTimedOut(timestampNanos))
-                        continue;
-                    iter.remove();
-                    dropped.incrementAndGet();
-                }
-
-                if (logger.isTraceEnabled())
-                {
-                    long duration = TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - timestampNanos);
-                    logger.trace("Expiration of {} took {}μs", getName(), duration);
-                }
-            }
-            finally
-            {
-                long backlogExpirationIntervalNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getOtcBacklogExpirationInterval());
-                backlogNextExpirationTime = timestampNanos + backlogExpirationIntervalNanos;
-                backlogExpirationActive.set(false);
-            }
-        }
-    }
-
-    /** messages that have not been retried yet */
-    private static class QueuedMessage implements Coalescable
-    {
-        final MessageOut<?> message;
-        final int id;
-        final long timestampNanos;
-        final boolean droppable;
-
-        QueuedMessage(MessageOut<?> message, int id, long timestampNanos)
-        {
-            this.message = message;
-            this.id = id;
-            this.timestampNanos = timestampNanos;
-            this.droppable = MessagingService.DROPPABLE_VERBS.contains(message.verb);
-        }
-
-        /** don't drop a non-droppable message just because it's timestamp is expired */
-        boolean isTimedOut(long nowNanos)
-        {
-            long messageTimeoutNanos = TimeUnit.MILLISECONDS.toNanos(message.getTimeout());
-            return droppable && nowNanos - timestampNanos  > messageTimeoutNanos;
-        }
-
-        boolean shouldRetry()
-        {
-            // retry all messages once
-            return true;
-        }
-
-        public long timestampNanos()
-        {
-            return timestampNanos;
-        }
-    }
-
-    private static class RetriedQueuedMessage extends QueuedMessage
-    {
-        RetriedQueuedMessage(QueuedMessage msg)
-        {
-            super(msg.message, msg.id, msg.timestampNanos);
-        }
-
-        boolean shouldRetry()
-        {
-            return false;
-        }
-    }
-
-    private static class InternodeAuthFailed extends Exception {}
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java b/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java
deleted file mode 100644
index 20a8da6..0000000
--- a/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java
+++ /dev/null
@@ -1,229 +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.net;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.nio.channels.SocketChannel;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.metrics.ConnectionMetrics;
-import org.apache.cassandra.security.SSLFactory;
-import org.apache.cassandra.utils.FBUtilities;
-
-public class OutboundTcpConnectionPool
-{
-    public static final long LARGE_MESSAGE_THRESHOLD =
-            Long.getLong(Config.PROPERTY_PREFIX + "otcp_large_message_threshold", 1024 * 64);
-
-    // pointer for the real Address.
-    private final InetAddress id;
-    private final CountDownLatch started;
-    public final OutboundTcpConnection smallMessages;
-    public final OutboundTcpConnection largeMessages;
-    public final OutboundTcpConnection gossipMessages;
-
-    // pointer to the reset Address.
-    private InetAddress resetEndpoint;
-    private ConnectionMetrics metrics;
-
-    // back-pressure state linked to this connection:
-    private final BackPressureState backPressureState;
-
-    OutboundTcpConnectionPool(InetAddress remoteEp, BackPressureState backPressureState)
-    {
-        id = remoteEp;
-        resetEndpoint = SystemKeyspace.getPreferredIP(remoteEp);
-        started = new CountDownLatch(1);
-
-        smallMessages = new OutboundTcpConnection(this, "Small");
-        largeMessages = new OutboundTcpConnection(this, "Large");
-        gossipMessages = new OutboundTcpConnection(this, "Gossip");
-
-        this.backPressureState = backPressureState;
-    }
-
-    /**
-     * returns the appropriate connection based on message type.
-     * returns null if a connection could not be established.
-     */
-    OutboundTcpConnection getConnection(MessageOut msg)
-    {
-        if (Stage.GOSSIP == msg.getStage())
-            return gossipMessages;
-        return msg.payloadSize(smallMessages.getTargetVersion()) > LARGE_MESSAGE_THRESHOLD
-               ? largeMessages
-               : smallMessages;
-    }
-
-    public BackPressureState getBackPressureState()
-    {
-        return backPressureState;
-    }
-
-    void reset()
-    {
-        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
-            conn.closeSocket(false);
-    }
-
-    public void resetToNewerVersion(int version)
-    {
-        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
-        {
-            if (version > conn.getTargetVersion())
-                conn.softCloseSocket();
-        }
-    }
-
-    /**
-     * reconnect to @param remoteEP (after the current message backlog is exhausted).
-     * Used by Ec2MultiRegionSnitch to force nodes in the same region to communicate over their private IPs.
-     * @param remoteEP
-     */
-    public void reset(InetAddress remoteEP)
-    {
-        SystemKeyspace.updatePreferredIP(id, remoteEP);
-        resetEndpoint = remoteEP;
-        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
-            conn.softCloseSocket();
-
-        // release previous metrics and create new one with reset address
-        metrics.release();
-        metrics = new ConnectionMetrics(resetEndpoint, this);
-    }
-
-    public long getTimeouts()
-    {
-       return metrics.timeouts.getCount();
-    }
-
-
-    public void incrementTimeout()
-    {
-        metrics.timeouts.mark();
-    }
-
-    public Socket newSocket() throws IOException
-    {
-        return newSocket(endPoint());
-    }
-
-    @SuppressWarnings("resource") // Closing the socket will close the underlying channel.
-    public static Socket newSocket(InetAddress endpoint) throws IOException
-    {
-        // zero means 'bind on any available port.'
-        if (isEncryptedChannel(endpoint))
-        {
-            return SSLFactory.getSocket(DatabaseDescriptor.getServerEncryptionOptions(), endpoint, DatabaseDescriptor.getSSLStoragePort());
-        }
-        else
-        {
-            SocketChannel channel = SocketChannel.open();
-            channel.connect(new InetSocketAddress(endpoint, DatabaseDescriptor.getStoragePort()));
-            return channel.socket();
-        }
-    }
-
-    public static int portFor(InetAddress endpoint)
-    {
-        return isEncryptedChannel(endpoint) ? DatabaseDescriptor.getSSLStoragePort() : DatabaseDescriptor.getStoragePort();
-    }
-
-    public InetAddress endPoint()
-    {
-        if (id.equals(FBUtilities.getBroadcastAddress()))
-            return FBUtilities.getLocalAddress();
-        return resetEndpoint;
-    }
-
-    public static boolean isEncryptedChannel(InetAddress address)
-    {
-        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        switch (DatabaseDescriptor.getServerEncryptionOptions().internode_encryption)
-        {
-            case none:
-                return false; // if nothing needs to be encrypted then return immediately.
-            case all:
-                break;
-            case dc:
-                if (snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddress())))
-                    return false;
-                break;
-            case rack:
-                // for rack then check if the DC's are the same.
-                if (snitch.getRack(address).equals(snitch.getRack(FBUtilities.getBroadcastAddress()))
-                        && snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddress())))
-                    return false;
-                break;
-        }
-        return true;
-    }
-
-    public void start()
-    {
-        smallMessages.start();
-        largeMessages.start();
-        gossipMessages.start();
-
-        metrics = new ConnectionMetrics(id, this);
-
-        started.countDown();
-    }
-
-    public void waitForStarted()
-    {
-        if (started.getCount() == 0)
-            return;
-
-        boolean error = false;
-        try
-        {
-            if (!started.await(1, TimeUnit.MINUTES))
-                error = true;
-        }
-        catch (InterruptedException e)
-        {
-            Thread.currentThread().interrupt();
-            error = true;
-        }
-        if (error)
-            throw new IllegalStateException(String.format("Connections to %s are not started!", id.getHostAddress()));
-    }
-
-    public void close()
-    {
-        // these null guards are simply for tests
-        if (largeMessages != null)
-            largeMessages.closeSocket(true);
-        if (smallMessages != null)
-            smallMessages.closeSocket(true);
-        if (gossipMessages != null)
-            gossipMessages.closeSocket(true);
-        if (metrics != null)
-            metrics.release();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java
new file mode 100644
index 0000000..f9fa07a
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.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.net.async;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import org.apache.cassandra.io.util.DataInputPlus;
+
+public class ByteBufDataInputPlus extends ByteBufInputStream implements DataInputPlus
+{
+    public ByteBufDataInputPlus(ByteBuf buffer)
+    {
+        super(buffer);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java
new file mode 100644
index 0000000..0473465
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java
@@ -0,0 +1,140 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+
+import com.google.common.base.Function;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufOutputStream;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.Memory;
+import org.apache.cassandra.io.util.UnbufferedDataOutputStreamPlus;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+import org.apache.cassandra.utils.vint.VIntCoding;
+
+/**
+ * A {@link DataOutputPlus} that uses a {@link ByteBuf} as a backing buffer. This class is completely thread unsafe and
+ * it is expected that the backing buffer is sized correctly for all the writes you want to do (or the buffer needs
+ * to be growable).
+ */
+public class ByteBufDataOutputPlus extends ByteBufOutputStream implements DataOutputPlus
+{
+    private final ByteBuf buffer;
+
+    /**
+     * ByteBuffer to use for defensive copies of direct {@link ByteBuffer}s - see {@link #write(ByteBuffer)}.
+     */
+    private final ByteBuffer hollowBuffer = MemoryUtil.getHollowDirectByteBuffer();
+
+    public ByteBufDataOutputPlus(ByteBuf buffer)
+    {
+        super(buffer);
+        this.buffer = buffer;
+    }
+
+    /**
+     * {@inheritDoc} - "write the buffer without modifying its position"
+     *
+     * Unfortunately, netty's {@link ByteBuf#writeBytes(ByteBuffer)} modifies the byteBuffer's position,
+     * and that is unsafe in our world wrt multithreading. Hence we need to be careful: reference the backing array
+     * on heap ByteBuffers, and use a reusable "hollow" ByteBuffer ({@link #hollowBuffer}) for direct ByteBuffers.
+     */
+    @Override
+    public void write(ByteBuffer byteBuffer) throws IOException
+    {
+        if (byteBuffer.hasArray())
+        {
+            write(byteBuffer.array(), byteBuffer.arrayOffset() + byteBuffer.position(), byteBuffer.remaining());
+        }
+        else
+        {
+            assert byteBuffer.isDirect();
+            MemoryUtil.duplicateDirectByteBuffer(byteBuffer, hollowBuffer);
+            buffer.writeBytes(hollowBuffer);
+        }
+    }
+
+    @Override
+    public void write(Memory memory, long offset, long length) throws IOException
+    {
+        for (ByteBuffer buffer : memory.asByteBuffers(offset, length))
+            write(buffer);
+    }
+
+    @Override
+    public <R> R applyToChannel(Function<WritableByteChannel, R> c) throws IOException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeVInt(long v) throws IOException
+    {
+        writeUnsignedVInt(VIntCoding.encodeZigZag64(v));
+    }
+
+    @Override
+    public void writeUnsignedVInt(long v) throws IOException
+    {
+        int size = VIntCoding.computeUnsignedVIntSize(v);
+        if (size == 1)
+        {
+            buffer.writeByte((byte) (v & 0xFF));
+            return;
+        }
+
+        buffer.writeBytes(VIntCoding.encodeVInt(v, size), 0, size);
+    }
+
+    @Override
+    public void write(int b) throws IOException
+    {
+        buffer.writeByte((byte) (b & 0xFF));
+    }
+
+    @Override
+    public void writeByte(int v) throws IOException
+    {
+        buffer.writeByte((byte) (v & 0xFF));
+    }
+
+    @Override
+    public void writeBytes(String s) throws IOException
+    {
+        for (int index = 0; index < s.length(); index++)
+            buffer.writeByte(s.charAt(index) & 0xFF);
+    }
+
+    @Override
+    public void writeChars(String s) throws IOException
+    {
+        for (int index = 0; index < s.length(); index++)
+            buffer.writeChar(s.charAt(index));
+    }
+
+    @Override
+    public void writeUTF(String s) throws IOException
+    {
+        UnbufferedDataOutputStreamPlus.writeUTF(s, this);
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[04/11] cassandra git commit: move streaming to use netty

Posted by ja...@apache.org.
move streaming to use netty

patch by jasobrown, reviewed by aweisberg for CASSANDRA-12229


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

Branch: refs/heads/trunk
Commit: fc92db2b9b56c143516026ba29cecdec37e286bb
Parents: 356dc3c
Author: Jason Brown <ja...@gmail.com>
Authored: Mon Apr 11 05:26:18 2016 -0700
Committer: Jason Brown <ja...@gmail.com>
Committed: Tue Aug 22 13:54:44 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 lib/compress-lzf-0.8.4.jar                      | Bin 25490 -> 0 bytes
 .../org/apache/cassandra/config/Config.java     |   6 -
 .../cassandra/config/DatabaseDescriptor.java    |  15 -
 .../exceptions/ChecksumMismatchException.java   |  34 ++
 .../io/compress/CompressionMetadata.java        |   4 +-
 .../cassandra/io/sstable/SSTableLoader.java     |   2 +-
 .../io/util/DataIntegrityMetadata.java          |  26 +
 .../net/IncomingStreamingConnection.java        | 104 ----
 .../net/async/ByteBufDataInputPlus.java         |  12 +
 .../net/async/ByteBufDataOutputStreamPlus.java  | 191 +++++++
 .../net/async/InboundHandshakeHandler.java      |  36 +-
 .../cassandra/net/async/NettyFactory.java       |  24 +-
 .../net/async/OutboundConnectionIdentifier.java |  21 +-
 .../net/async/OutboundHandshakeHandler.java     |   9 +-
 .../async/RebufferingByteBufDataInputPlus.java  | 250 +++++++++
 .../apache/cassandra/security/SSLFactory.java   |  49 --
 .../cassandra/service/StorageService.java       |  11 -
 .../cassandra/service/StorageServiceMBean.java  |   3 -
 .../cassandra/streaming/ConnectionHandler.java  | 428 ----------------
 .../streaming/DefaultConnectionFactory.java     | 122 +++--
 .../streaming/StreamConnectionFactory.java      |  11 +-
 .../cassandra/streaming/StreamCoordinator.java  |  22 +-
 .../cassandra/streaming/StreamManager.java      |  24 +-
 .../apache/cassandra/streaming/StreamPlan.java  |   2 +-
 .../cassandra/streaming/StreamReader.java       |  25 +-
 .../streaming/StreamReceiveException.java       |  36 ++
 .../cassandra/streaming/StreamReceiveTask.java  |   1 +
 .../cassandra/streaming/StreamResultFuture.java |  32 +-
 .../cassandra/streaming/StreamSession.java      | 396 ++++++++-------
 .../cassandra/streaming/StreamTransferTask.java |  10 +-
 .../cassandra/streaming/StreamWriter.java       | 115 +++--
 .../streaming/StreamingMessageSender.java       |  34 ++
 .../async/NettyStreamingMessageSender.java      | 508 +++++++++++++++++++
 .../async/StreamCompressionSerializer.java      | 133 +++++
 .../async/StreamingInboundHandler.java          | 268 ++++++++++
 .../cassandra/streaming/async/package-info.java |  71 +++
 .../ByteBufCompressionDataOutputStreamPlus.java |  76 +++
 .../compress/CompressedInputStream.java         | 225 ++++----
 .../compress/CompressedStreamReader.java        |  17 +-
 .../compress/CompressedStreamWriter.java        |  25 +-
 .../compress/StreamCompressionInputStream.java  |  78 +++
 .../streaming/messages/CompleteMessage.java     |  10 +-
 .../streaming/messages/FileMessageHeader.java   |  38 +-
 .../streaming/messages/IncomingFileMessage.java |  30 +-
 .../streaming/messages/KeepAliveMessage.java    |   9 +-
 .../streaming/messages/OutgoingFileMessage.java |  28 +-
 .../streaming/messages/PrepareAckMessage.java   |  57 +++
 .../streaming/messages/PrepareMessage.java      |  93 ----
 .../messages/PrepareSynAckMessage.java          |  80 +++
 .../streaming/messages/PrepareSynMessage.java   |  98 ++++
 .../streaming/messages/ReceivedMessage.java     |  11 +-
 .../streaming/messages/RetryMessage.java        |  71 ---
 .../messages/SessionFailedMessage.java          |  10 +-
 .../streaming/messages/StreamInitMessage.java   |  73 +--
 .../streaming/messages/StreamMessage.java       |  58 +--
 .../tools/BulkLoadConnectionFactory.java        |  32 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   7 -
 .../cassandra/tools/nodetool/GetTimeout.java    |   2 +-
 .../org/apache/cassandra/utils/UUIDGen.java     |   8 +-
 .../cassandra/streaming/LongStreamingTest.java  |  34 +-
 .../cassandra/cql3/PreparedStatementsTest.java  |   2 +-
 .../util/RewindableDataInputStreamPlusTest.java |   2 +-
 .../net/async/HandshakeHandlersTest.java        |   4 +-
 .../net/async/InboundHandshakeHandlerTest.java  |   8 +-
 .../async/OutboundMessagingConnectionTest.java  |  45 --
 .../RebufferingByteBufDataInputPlusTest.java    | 126 +++++
 .../net/async/TestScheduledFuture.java          |  66 +++
 .../apache/cassandra/service/RemoveTest.java    |   3 +-
 .../streaming/StreamTransferTaskTest.java       |   7 +-
 .../streaming/StreamingTransferTest.java        |  28 -
 .../async/NettyStreamingMessageSenderTest.java  | 202 ++++++++
 .../async/StreamCompressionSerializerTest.java  | 135 +++++
 .../async/StreamingInboundHandlerTest.java      | 168 ++++++
 .../compression/CompressedInputStreamTest.java  |  10 +-
 75 files changed, 3508 insertions(+), 1504 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f2e643e..a14e390 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * use netty for streaming (CASSANDRA-12229)
  * Use netty for internode messaging (CASSANDRA-8457)
  * Add bytes repaired/unrepaired to nodetool tablestats (CASSANDRA-13774)
  * Don't delete incremental repair sessions if they still have sstables (CASSANDRA-13758)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/lib/compress-lzf-0.8.4.jar
----------------------------------------------------------------------
diff --git a/lib/compress-lzf-0.8.4.jar b/lib/compress-lzf-0.8.4.jar
deleted file mode 100644
index a712c24..0000000
Binary files a/lib/compress-lzf-0.8.4.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 77d5bf4..537cf39 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -97,12 +97,6 @@ public class Config
 
     public volatile long truncate_request_timeout_in_ms = 60000L;
 
-    /**
-     * @deprecated use {@link #streaming_keep_alive_period_in_secs} instead
-     */
-    @Deprecated
-    public int streaming_socket_timeout_in_ms = 86400000; //24 hours
-
     public Integer streaming_connections_per_host = 1;
     public Integer streaming_keep_alive_period_in_secs = 300; //5 minutes
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 53bac93..302a528 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -2060,21 +2060,6 @@ public class DatabaseDescriptor
         conf.counter_cache_keys_to_save = counterCacheKeysToSave;
     }
 
-    public static void setStreamingSocketTimeout(int value)
-    {
-        conf.streaming_socket_timeout_in_ms = value;
-    }
-
-    /**
-     * @deprecated use {@link #getStreamingKeepAlivePeriod()} instead
-     * @return streaming_socket_timeout_in_ms property
-     */
-    @Deprecated
-    public static int getStreamingSocketTimeout()
-    {
-        return conf.streaming_socket_timeout_in_ms;
-    }
-
     public static int getStreamingKeepAlivePeriod()
     {
         return conf.streaming_keep_alive_period_in_secs;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/exceptions/ChecksumMismatchException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/ChecksumMismatchException.java b/src/java/org/apache/cassandra/exceptions/ChecksumMismatchException.java
new file mode 100644
index 0000000..a76c46c
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/ChecksumMismatchException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.exceptions;
+
+import java.io.IOException;
+
+public class ChecksumMismatchException extends IOException
+{
+    public ChecksumMismatchException()
+    {
+        super();
+    }
+
+    public ChecksumMismatchException(String s)
+    {
+        super(s);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index 6c1849f..8ac6589 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -145,7 +145,9 @@ public class CompressionMetadata
         this.chunkOffsetsSize = chunkOffsets.size();
     }
 
-    private CompressionMetadata(String filePath, CompressionParams parameters, SafeMemory offsets, long offsetsSize, long dataLength, long compressedLength)
+    // do not call this constructor directly, unless used in testing
+    @VisibleForTesting
+    public CompressionMetadata(String filePath, CompressionParams parameters, Memory offsets, long offsetsSize, long dataLength, long compressedLength)
     {
         this.indexFilePath = filePath;
         this.parameters = parameters;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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 ff47bec..dc56520 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -208,7 +208,7 @@ public class SSTableLoader implements StreamEventHandler
         for (SSTableReader sstable : sstables)
         {
             sstable.selfRef().release();
-            assert sstable.selfRef().globalCount() == 0;
+            assert sstable.selfRef().globalCount() == 0 : String.format("for sstable = %s, ref count = %d", sstable, sstable.selfRef().globalCount());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java b/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
index 91b189d..277b359 100644
--- a/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
+++ b/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
@@ -20,9 +20,12 @@ package org.apache.cassandra.io.util;
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.zip.CheckedInputStream;
 import java.util.zip.Checksum;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.utils.ChecksumType;
@@ -57,6 +60,15 @@ public class DataIntegrityMetadata
             chunkSize = reader.readInt();
         }
 
+        @VisibleForTesting
+        protected ChecksumValidator(ChecksumType checksumType, RandomAccessReader reader, int chunkSize)
+        {
+            this.checksumType = checksumType;
+            this.reader = reader;
+            this.dataFilename = null;
+            this.chunkSize = chunkSize;
+        }
+
         public void seek(long offset)
         {
             long start = chunkStart(offset);
@@ -77,6 +89,20 @@ public class DataIntegrityMetadata
                 throw new IOException("Corrupted File : " + dataFilename);
         }
 
+        /**
+         * validates the checksum with the bytes from the specified buffer.
+         *
+         * Upon return, the buffer's position will
+         * be updated to its limit; its limit will not have been changed.
+         */
+        public void validate(ByteBuffer buffer) throws IOException
+        {
+            int current = (int) checksumType.of(buffer);
+            int actual = reader.readInt();
+            if (current != actual)
+                throw new IOException("Corrupted File : " + dataFilename);
+        }
+
         public void close()
         {
             reader.close();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
deleted file mode 100644
index 8db5fcb..0000000
--- a/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
+++ /dev/null
@@ -1,104 +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.net;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.Socket;
-import java.util.Set;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.streaming.StreamResultFuture;
-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 implements Closeable
-{
-    private static final Logger logger = LoggerFactory.getLogger(IncomingStreamingConnection.class);
-
-    private final int version;
-    public final Socket socket;
-    private final Set<Closeable> group;
-
-    public IncomingStreamingConnection(int version, Socket socket, Set<Closeable> group)
-    {
-        super("STREAM-INIT-" + socket.getRemoteSocketAddress());
-        this.version = version;
-        this.socket = socket;
-        this.group = group;
-    }
-
-    @Override
-    @SuppressWarnings("resource") // Not closing constructed DataInputPlus's as the stream needs to remain open.
-    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, StreamMessage.CURRENT_VERSION));
-
-            DataInputPlus input = new DataInputStreamPlus(socket.getInputStream());
-            StreamInitMessage init = StreamInitMessage.serializer.deserialize(input, version);
-
-            //Set SO_TIMEOUT on follower side
-            if (!init.isForOutgoing)
-                socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
-
-            // The initiator makes two connections, one for incoming and one for outgoing.
-            // The receiving side distinguish two connections by looking at StreamInitMessage#isForOutgoing.
-            // Note: we cannot use the same socket for incoming and outgoing streams because we want to
-            // parallelize said streams and the socket is blocking, so we might deadlock.
-            StreamResultFuture.initReceivingSide(init.sessionIndex, init.planId, init.streamOperation, init.from, this, init.isForOutgoing, version, init.keepSSTableLevel, init.pendingRepair, init.previewKind);
-        }
-        catch (Throwable t)
-        {
-            logger.error("Error while reading from socket from {}.", socket.getRemoteSocketAddress(), t);
-            close();
-        }
-    }
-
-    @Override
-    public void close()
-    {
-        try
-        {
-            if (!socket.isClosed())
-            {
-                socket.close();
-            }
-        }
-        catch (IOException e)
-        {
-            logger.debug("Error closing socket", e);
-        }
-        finally
-        {
-            group.remove(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java
index f9fa07a..23e532c 100644
--- a/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java
+++ b/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java
@@ -24,8 +24,20 @@ import org.apache.cassandra.io.util.DataInputPlus;
 
 public class ByteBufDataInputPlus extends ByteBufInputStream implements DataInputPlus
 {
+    /**
+     * The parent class does not expose the buffer to derived classes, so we need
+     * to stash a reference here so it can be exposed via {@link #buffer()}.
+     */
+    private final ByteBuf buf;
+
     public ByteBufDataInputPlus(ByteBuf buffer)
     {
         super(buffer);
+        this.buf = buffer;
+    }
+
+    public ByteBuf buffer()
+    {
+        return buf;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java
new file mode 100644
index 0000000..3a544e4
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java
@@ -0,0 +1,191 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.Future;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.streaming.StreamSession;
+
+/**
+ * A {@link DataOutputStreamPlus} that writes to a {@link ByteBuf}. The novelty here is that all writes
+ * actually get written in to a {@link ByteBuffer} that shares a backing buffer with a {@link ByteBuf}.
+ * The trick to do that is allocate the ByteBuf, get a ByteBuffer from it by calling {@link ByteBuf#nioBuffer()},
+ * and passing that to the super class as {@link #buffer}. When the {@link #buffer} is full or {@link #doFlush(int)}
+ * is invoked, the {@link #currentBuf} is published to the netty channel.
+ */
+public class ByteBufDataOutputStreamPlus extends BufferedDataOutputStreamPlus
+{
+    private final StreamSession session;
+    private final Channel channel;
+    private final int bufferSize;
+
+    /**
+     * Tracks how many bytes we've written to the netty channel. This more or less follows the channel's
+     * high/low water marks and ultimately the 'writablility' status of the channel. Unfortunately there's
+     * no notification mechanism that can poke a producer to let it know when the channel becomes writable
+     * (after it was unwritable); hence, the use of a {@link Semaphore}.
+     */
+    private final Semaphore channelRateLimiter;
+
+    /**
+     * This *must* be the owning {@link ByteBuf} for the {@link BufferedDataOutputStreamPlus#buffer}
+     */
+    private ByteBuf currentBuf;
+
+    private ByteBufDataOutputStreamPlus(StreamSession session, Channel channel, ByteBuf buffer, int bufferSize)
+    {
+        super(buffer.nioBuffer(0, bufferSize));
+        this.session = session;
+        this.channel = channel;
+        this.currentBuf = buffer;
+        this.bufferSize = bufferSize;
+
+        channelRateLimiter = new Semaphore(channel.config().getWriteBufferHighWaterMark(), true);
+    }
+
+    @Override
+    protected WritableByteChannel newDefaultChannel()
+    {
+        return new WritableByteChannel()
+        {
+            @Override
+            public int write(ByteBuffer src) throws IOException
+            {
+                assert src == buffer;
+                int size = src.position();
+                doFlush(size);
+                return size;
+            }
+
+            @Override
+            public boolean isOpen()
+            {
+                return channel.isOpen();
+            }
+
+            @Override
+            public void close()
+            {   }
+        };
+    }
+
+    public static ByteBufDataOutputStreamPlus create(StreamSession session, Channel channel, int bufferSize)
+    {
+        ByteBuf buf = channel.alloc().directBuffer(bufferSize, bufferSize);
+        return new ByteBufDataOutputStreamPlus(session, channel, buf, bufferSize);
+    }
+
+    /**
+     * Writes the incoming buffer directly to the backing {@link #channel}, without copying to the intermediate {@link #buffer}.
+     */
+    public ChannelFuture writeToChannel(ByteBuf buf) throws IOException
+    {
+        doFlush(buffer.position());
+
+        int byteCount = buf.readableBytes();
+        if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, byteCount, 5, TimeUnit.MINUTES))
+            throw new IOException("outbound channel was not writable");
+
+        // the (possibly naive) assumption that we should always flush after each incoming buf
+        ChannelFuture channelFuture = channel.writeAndFlush(buf);
+        channelFuture.addListener(future -> handleBuffer(future, byteCount));
+        return channelFuture;
+    }
+
+    /**
+     * Writes the incoming buffer directly to the backing {@link #channel}, without copying to the intermediate {@link #buffer}.
+     * The incoming buffer will be automatically released when the netty channel invokes the listeners of success/failure to
+     * send the buffer.
+     */
+    public ChannelFuture writeToChannel(ByteBuffer buffer) throws IOException
+    {
+        ChannelFuture channelFuture = writeToChannel(Unpooled.wrappedBuffer(buffer));
+        channelFuture.addListener(future -> FileUtils.clean(buffer));
+        return channelFuture;
+    }
+
+    @Override
+    protected void doFlush(int count) throws IOException
+    {
+        // flush the current backing write buffer only if there's any pending data
+        if (buffer.position() > 0 && channel.isOpen())
+        {
+            int byteCount = buffer.position();
+            currentBuf.writerIndex(byteCount);
+
+            if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, byteCount, 2, TimeUnit.MINUTES))
+                throw new IOException("outbound channel was not writable");
+
+            channel.writeAndFlush(currentBuf).addListener(future -> handleBuffer(future, byteCount));
+            currentBuf = channel.alloc().directBuffer(bufferSize, bufferSize);
+            buffer = currentBuf.nioBuffer(0, bufferSize);
+        }
+    }
+
+    /**
+     * Handles the result of publishing a buffer to the channel.
+     *
+     * Note: this will be executed on the event loop.
+     */
+    private void handleBuffer(Future<? super Void> future, int bytesWritten)
+    {
+        channelRateLimiter.release(bytesWritten);
+
+        if (!future.isSuccess() && channel.isOpen())
+            session.onError(future.cause());
+    }
+
+    public ByteBufAllocator getAllocator()
+    {
+        return channel.alloc();
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Flush any last buffered (if the channel is open), and release any buffers. *Not* responsible for closing
+     * the netty channel as we might use it again for transferring more files.
+     *
+     * Note: should be called on the producer thread, not the netty event loop.
+     */
+    @Override
+    public void close() throws IOException
+    {
+        doFlush(0);
+        if (currentBuf.refCnt() > 0)
+            currentBuf.release();
+        currentBuf = null;
+        buffer = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java b/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
index 5ea03dc..7a8303c 100644
--- a/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
+++ b/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
@@ -14,6 +14,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import io.netty.buffer.ByteBuf;
+import io.netty.channel.AdaptiveRecvByteBufAllocator;
 import io.netty.channel.ChannelFutureListener;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelPipeline;
@@ -25,6 +26,8 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage;
 import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage;
 import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage;
+import org.apache.cassandra.streaming.async.StreamingInboundHandler;
+import org.apache.cassandra.streaming.messages.StreamMessage;
 
 /**
  * 'Server'-side component that negotiates the internode handshake when establishing a new connection.
@@ -36,13 +39,13 @@ class InboundHandshakeHandler extends ByteToMessageDecoder
 {
     private static final Logger logger = LoggerFactory.getLogger(NettyFactory.class);
 
-    enum State { START, AWAITING_HANDSHAKE_BEGIN, AWAIT_STREAM_START_RESPONSE, AWAIT_MESSAGING_START_RESPONSE, MESSAGING_HANDSHAKE_COMPLETE, HANDSHAKE_FAIL }
+    enum State { START, AWAITING_HANDSHAKE_BEGIN, AWAIT_MESSAGING_START_RESPONSE, HANDSHAKE_COMPLETE, HANDSHAKE_FAIL }
 
     private State state;
 
     private final IInternodeAuthenticator authenticator;
-    private boolean hasAuthenticated;
 
+    private boolean hasAuthenticated;
     /**
      * The peer's declared messaging version.
      */
@@ -160,9 +163,16 @@ class InboundHandshakeHandler extends ByteToMessageDecoder
 
         if (msg.mode == NettyFactory.Mode.STREAMING)
         {
-            // TODO fill in once streaming is moved to netty
-            ctx.close();
-            return State.AWAIT_STREAM_START_RESPONSE;
+            // 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)
+            {
+                logger.warn("Received stream using protocol version %d (my version %d). Terminating connection", version, MessagingService.current_version);
+                ctx.close();
+                return State.HANDSHAKE_FAIL;
+            }
+
+            setupStreamingPipeline(ctx, version);
+            return State.HANDSHAKE_COMPLETE;
         }
         else
         {
@@ -195,6 +205,18 @@ class InboundHandshakeHandler extends ByteToMessageDecoder
         }
     }
 
+    private void setupStreamingPipeline(ChannelHandlerContext ctx, int protocolVersion)
+    {
+        ChannelPipeline pipeline = ctx.pipeline();
+        InetSocketAddress address = (InetSocketAddress) ctx.channel().remoteAddress();
+        pipeline.addLast(NettyFactory.instance.streamingGroup, "streamInbound", new StreamingInboundHandler(address, protocolVersion, null));
+        pipeline.remove(this);
+
+        // pass a custom recv ByteBuf allocator to the channel. the default recv ByteBuf size is 1k, but in streaming we're
+        // dealing with large bulk blocks of data, let's default to larger sizes
+        ctx.channel().config().setRecvByteBufAllocator(new AdaptiveRecvByteBufAllocator(1 << 8, 1 << 13, 1 << 16));
+    }
+
     /**
      * Handles the third (and last) message in the internode messaging handshake protocol. Grabs the protocol version and
      * IP addr the peer wants to use.
@@ -227,7 +249,7 @@ class InboundHandshakeHandler extends ByteToMessageDecoder
         logger.trace("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from));
 
         setupMessagingPipeline(ctx.pipeline(), from, compressed, version);
-        return State.MESSAGING_HANDSHAKE_COMPLETE;
+        return State.HANDSHAKE_COMPLETE;
     }
 
     @VisibleForTesting
@@ -245,7 +267,7 @@ class InboundHandshakeHandler extends ByteToMessageDecoder
     {
         // we're not really racing on the handshakeTimeout as we're in the event loop,
         // but, hey, defensive programming is beautiful thing!
-        if (state == State.MESSAGING_HANDSHAKE_COMPLETE || (handshakeTimeout != null && handshakeTimeout.isCancelled()))
+        if (state == State.HANDSHAKE_COMPLETE || (handshakeTimeout != null && handshakeTimeout.isCancelled()))
             return;
 
         state = State.HANDSHAKE_FAIL;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/net/async/NettyFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/NettyFactory.java b/src/java/org/apache/cassandra/net/async/NettyFactory.java
index 13d8810..762c39b 100644
--- a/src/java/org/apache/cassandra/net/async/NettyFactory.java
+++ b/src/java/org/apache/cassandra/net/async/NettyFactory.java
@@ -40,6 +40,7 @@ import io.netty.util.concurrent.DefaultThreadFactory;
 import io.netty.util.concurrent.EventExecutor;
 import io.netty.util.internal.logging.InternalLoggerFactory;
 import io.netty.util.internal.logging.Slf4JLoggerFactory;
+
 import net.jpountz.lz4.LZ4Factory;
 import net.jpountz.xxhash.XXHashFactory;
 import org.apache.cassandra.auth.IInternodeAuthenticator;
@@ -69,12 +70,18 @@ public final class NettyFactory
 
     private static final int LZ4_HASH_SEED = 0x9747b28c;
 
+    /**
+     * Default seed value for xxhash.
+     */
+    public static final int XXHASH_DEFAULT_SEED = 0x9747b28c;
+
     public enum Mode { MESSAGING, STREAMING }
 
     private static final String SSL_CHANNEL_HANDLER_NAME = "ssl";
-    static final String INBOUND_COMPRESSOR_HANDLER_NAME = "inboundCompressor";
-    static final String OUTBOUND_COMPRESSOR_HANDLER_NAME = "outboundCompressor";
-    private static final String HANDSHAKE_HANDLER_NAME = "handshakeHandler";
+    public static final String INBOUND_COMPRESSOR_HANDLER_NAME = "inboundCompressor";
+    public static final String OUTBOUND_COMPRESSOR_HANDLER_NAME = "outboundCompressor";
+    public static final String HANDSHAKE_HANDLER_NAME = "handshakeHandler";
+    public static final String INBOUND_STREAM_HANDLER_NAME = "inboundStreamHandler";
 
     /** a useful addition for debugging; simply set to true to get more data in your logs */
     private static final boolean WIRETRACE = false;
@@ -113,6 +120,7 @@ public final class NettyFactory
 
     private final EventLoopGroup inboundGroup;
     private final EventLoopGroup outboundGroup;
+    public final EventLoopGroup streamingGroup;
 
     /**
      * Constructor that allows modifying the {@link NettyFactory#useEpoll} for testing purposes. Otherwise, use the
@@ -126,6 +134,7 @@ public final class NettyFactory
                                         "MessagingService-NettyAcceptor-Threads", false);
         inboundGroup = getEventLoopGroup(useEpoll, FBUtilities.getAvailableProcessors(), "MessagingService-NettyInbound-Threads", false);
         outboundGroup = getEventLoopGroup(useEpoll, FBUtilities.getAvailableProcessors(), "MessagingService-NettyOutbound-Threads", true);
+        streamingGroup = getEventLoopGroup(useEpoll, FBUtilities.getAvailableProcessors(), "Streaming-Netty-Threads", false);
     }
 
     /**
@@ -257,7 +266,8 @@ public final class NettyFactory
                 SslContext sslContext = SSLFactory.getSslContext(encryptionOptions, true, true);
                 SslHandler sslHandler = sslContext.newHandler(channel.alloc());
                 logger.trace("creating inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
-                pipeline.addFirst(SSL_CHANNEL_HANDLER_NAME, sslHandler);            }
+                pipeline.addFirst(SSL_CHANNEL_HANDLER_NAME, sslHandler);
+            }
 
             if (WIRETRACE)
                 pipeline.addLast("logger", new LoggingHandler(LogLevel.INFO));
@@ -279,13 +289,14 @@ public final class NettyFactory
      * Create the {@link Bootstrap} for connecting to a remote peer. This method does <b>not</b> attempt to connect to the peer,
      * and thus does not block.
      */
+    @VisibleForTesting
     public Bootstrap createOutboundBootstrap(OutboundConnectionParams params)
     {
         logger.debug("creating outbound bootstrap to peer {}, compression: {}, encryption: {}, coalesce: {}", params.connectionId.connectionAddress(),
                      params.compress, encryptionLogStatement(params.encryptionOptions),
                      params.coalescingStrategy.isPresent() ? params.coalescingStrategy.get() : CoalescingStrategies.Strategy.DISABLED);
-        Class<? extends Channel>  transport = useEpoll ? EpollSocketChannel.class : NioSocketChannel.class;
-        Bootstrap bootstrap = new Bootstrap().group(outboundGroup)
+        Class<? extends Channel> transport = useEpoll ? EpollSocketChannel.class : NioSocketChannel.class;
+        Bootstrap bootstrap = new Bootstrap().group(params.mode == Mode.MESSAGING ? outboundGroup : streamingGroup)
                               .channel(transport)
                               .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 2000)
                               .option(ChannelOption.SO_KEEPALIVE, true)
@@ -349,6 +360,7 @@ public final class NettyFactory
         acceptGroup.shutdownGracefully();
         outboundGroup.shutdownGracefully();
         inboundGroup.shutdownGracefully();
+        streamingGroup.shutdownGracefully();
     }
 
     static Lz4FrameEncoder createLz4Encoder(int protocolVersion)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java b/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
index 24dc5ff..c834bd4 100644
--- a/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
+++ b/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
@@ -32,7 +32,7 @@ public class OutboundConnectionIdentifier
 {
     enum ConnectionType
     {
-        GOSSIP, LARGE_MESSAGE, SMALL_MESSAGE
+        GOSSIP, LARGE_MESSAGE, SMALL_MESSAGE, STREAM
     }
 
     /**
@@ -99,6 +99,15 @@ public class OutboundConnectionIdentifier
     }
 
     /**
+     * Creates an identifier for a gossip connection and using the remote "identifying" address as its connection
+     * address.
+     */
+    public static OutboundConnectionIdentifier stream(InetSocketAddress localAddr, InetSocketAddress remoteAddr)
+    {
+        return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.STREAM);
+    }
+
+    /**
      * Returns a newly created connection identifier to the same remote that this identifier, but using the provided
      * address as connection address.
      *
@@ -106,7 +115,7 @@ public class OutboundConnectionIdentifier
      * @return a newly created connection identifier that differs from this one only by using {@code remoteConnectionAddr}
      * as connection address to the remote.
      */
-    OutboundConnectionIdentifier withNewConnectionAddress(InetSocketAddress remoteConnectionAddr)
+    public OutboundConnectionIdentifier withNewConnectionAddress(InetSocketAddress remoteConnectionAddr)
     {
         return new OutboundConnectionIdentifier(localAddr, remoteAddr, remoteConnectionAddr, connectionType);
     }
@@ -114,7 +123,7 @@ public class OutboundConnectionIdentifier
     /**
      * The local node address.
      */
-    InetAddress local()
+    public InetAddress local()
     {
         return localAddr.getAddress();
     }
@@ -122,7 +131,7 @@ public class OutboundConnectionIdentifier
     /**
      * The remote node identifying address (the one to use for anything else than connecting to the node).
      */
-    InetSocketAddress remoteAddress()
+    public InetSocketAddress remoteAddress()
     {
         return remoteAddr;
     }
@@ -130,7 +139,7 @@ public class OutboundConnectionIdentifier
     /**
      * The remote node identifying address (the one to use for anything else than connecting to the node).
      */
-    InetAddress remote()
+    public  InetAddress remote()
     {
         return remoteAddr.getAddress();
     }
@@ -138,7 +147,7 @@ public class OutboundConnectionIdentifier
     /**
      * The remote node connection address (the one to use to actually connect to the remote, and only that).
      */
-    InetSocketAddress connectionAddress()
+    public InetSocketAddress connectionAddress()
     {
         return remoteConnectionAddr;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java b/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
index 703549a..c555bed 100644
--- a/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
+++ b/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
@@ -36,6 +36,7 @@ import io.netty.channel.ChannelPipeline;
 import io.netty.handler.codec.ByteToMessageDecoder;
 import io.netty.handler.timeout.IdleStateHandler;
 import io.netty.util.concurrent.Future;
+
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage;
@@ -95,7 +96,9 @@ public class OutboundHandshakeHandler extends ByteToMessageDecoder
     /**
      * {@inheritDoc}
      *
-     * Invoked when the channel is made active, and sends out the {@link FirstHandshakeMessage}
+     * Invoked when the channel is made active, and sends out the {@link FirstHandshakeMessage}.
+     * In the case of streaming, we do not require a full bi-directional handshake; the initial message,
+     * containing the streaming protocol version, is all that is required.
      */
     @Override
     public void channelActive(final ChannelHandlerContext ctx) throws Exception
@@ -103,6 +106,10 @@ public class OutboundHandshakeHandler extends ByteToMessageDecoder
         FirstHandshakeMessage msg = new FirstHandshakeMessage(messagingVersion, mode, params.compress);
         logger.trace("starting handshake with peer {}, msg = {}", connectionId.connectionAddress(), msg);
         ctx.writeAndFlush(msg.encode(ctx.alloc())).addListener(future -> firstHandshakeMessageListener(future, ctx));
+
+        if (mode == NettyFactory.Mode.STREAMING)
+            ctx.pipeline().remove(this);
+
         ctx.fireChannelActive();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java
new file mode 100644
index 0000000..580bc03
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java
@@ -0,0 +1,250 @@
+/*
+ * 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.async;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelConfig;
+import io.netty.util.ReferenceCountUtil;
+import org.apache.cassandra.io.util.RebufferingInputStream;
+
+public class RebufferingByteBufDataInputPlus extends RebufferingInputStream implements ReadableByteChannel
+{
+    /**
+     * The parent, or owning, buffer of the current buffer being read from ({@link super#buffer}).
+     */
+    private ByteBuf currentBuf;
+
+    private final BlockingQueue<ByteBuf> queue;
+
+    /**
+     * The count of live bytes in all {@link ByteBuf}s held by this instance.
+     */
+    private final AtomicInteger queuedByteCount;
+
+    private final int lowWaterMark;
+    private final int highWaterMark;
+    private final ChannelConfig channelConfig;
+
+    private volatile boolean closed;
+
+    public RebufferingByteBufDataInputPlus(int lowWaterMark, int highWaterMark, ChannelConfig channelConfig)
+    {
+        super(Unpooled.EMPTY_BUFFER.nioBuffer());
+
+        if (lowWaterMark > highWaterMark)
+            throw new IllegalArgumentException(String.format("low water mark is greater than high water mark: %d vs %d", lowWaterMark, highWaterMark));
+
+        currentBuf = Unpooled.EMPTY_BUFFER;
+        this.lowWaterMark = lowWaterMark;
+        this.highWaterMark = highWaterMark;
+        this.channelConfig = channelConfig;
+        queue = new LinkedBlockingQueue<>();
+        queuedByteCount = new AtomicInteger();
+    }
+
+    /**
+     * Append a {@link ByteBuf} to the end of the einternal queue.
+     *
+     * Note: it's expected this method is invoked on the netty event loop.
+     */
+    public void append(ByteBuf buf) throws IllegalStateException
+    {
+        assert buf != null : "buffer cannot be null";
+
+        if (closed)
+        {
+            ReferenceCountUtil.release(buf);
+            throw new IllegalStateException("stream is already closed, so cannot add another buffer");
+        }
+
+        // this slightly undercounts the live count as it doesn't include the currentBuf's size.
+        // that's ok as the worst we'll do is allow another buffer in and add it to the queue,
+        // and that point we'll disable auto-read. this is a tradeoff versus making some other member field
+        // atomic or volatile.
+        int queuedCount = queuedByteCount.addAndGet(buf.readableBytes());
+        if (channelConfig.isAutoRead() && queuedCount > highWaterMark)
+            channelConfig.setAutoRead(false);
+
+        queue.add(buf);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Release open buffers and poll the {@link #queue} for more data.
+     * <p>
+     * This is best, and more or less expected, to be invoked on a consuming thread (not the event loop)
+     * becasue if we block on the queue we can't fill it on the event loop (as that's where the buffers are coming from).
+     */
+    @Override
+    protected void reBuffer() throws IOException
+    {
+        currentBuf.release();
+        buffer = null;
+        currentBuf = null;
+
+        // possibly re-enable auto-read, *before* blocking on the queue, because if we block on the queue
+        // without enabling auto-read we'll block forever :(
+        if (!channelConfig.isAutoRead() && queuedByteCount.get() < lowWaterMark)
+            channelConfig.setAutoRead(true);
+
+        try
+        {
+            currentBuf = queue.take();
+            int bytes;
+            // if we get an explicitly empty buffer, we treat that as an indicator that the input is closed
+            if (currentBuf == null || (bytes = currentBuf.readableBytes()) == 0)
+            {
+                releaseResources();
+                throw new EOFException();
+            }
+
+            buffer = currentBuf.nioBuffer(currentBuf.readerIndex(), bytes);
+            assert buffer.remaining() == bytes;
+            queuedByteCount.addAndGet(-bytes);
+            return;
+        }
+        catch (InterruptedException ie)
+        {
+            // nop - ignore
+        }
+    }
+
+    @Override
+    public int read(ByteBuffer dst) throws IOException
+    {
+        int readLength = dst.remaining();
+        int remaining = readLength;
+
+        while (remaining > 0)
+        {
+            if (closed)
+                throw new EOFException();
+
+            if (!buffer.hasRemaining())
+                reBuffer();
+            int copyLength = Math.min(remaining, buffer.remaining());
+
+            int originalLimit = buffer.limit();
+            buffer.limit(buffer.position() + copyLength);
+            dst.put(buffer);
+            buffer.limit(originalLimit);
+            remaining -= copyLength;
+        }
+
+        return readLength;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * As long as this method is invoked on the consuming thread the returned value will be accurate.
+     */
+    @Override
+    public int available() throws EOFException
+    {
+        if (closed)
+            throw new EOFException();
+
+       final  int availableBytes = queuedByteCount.get() + (buffer != null ? buffer.remaining() : 0);
+
+        if (!channelConfig.isAutoRead() && availableBytes < lowWaterMark)
+            channelConfig.setAutoRead(true);
+
+        return availableBytes;
+    }
+
+    @Override
+    public boolean isOpen()
+    {
+        return !closed;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Note: This should invoked on the consuming thread.
+     */
+    @Override
+    public void close()
+    {
+        closed = true;
+        releaseResources();
+    }
+
+    private void releaseResources()
+    {
+        if (currentBuf != null)
+        {
+            if (currentBuf.refCnt() > 0)
+                currentBuf.release(currentBuf.refCnt());
+            currentBuf = null;
+            buffer = null;
+        }
+
+        ByteBuf buf;
+        while ((buf = queue.poll()) != null && buf.refCnt() > 0)
+            buf.release(buf.refCnt());
+    }
+
+    /**
+     * Mark this stream as closed, but do not release any of the resources.
+     *
+     * Note: this is best to be called from the producer thread.
+     */
+    public void markClose()
+    {
+        if (!closed)
+        {
+            closed = true;
+            queue.add(Unpooled.EMPTY_BUFFER);
+        }
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Note: this is best to be called from the consumer thread.
+     */
+    @Override
+    public String toString()
+    {
+        return new StringBuilder(128).append("RebufferingByteBufDataInputPlus: currentBuf = ").append(currentBuf)
+                                  .append(" (super.buffer = ").append(buffer).append(')')
+                                  .append(", queuedByteCount = ").append(queuedByteCount)
+                                  .append(", queue buffers = ").append(queue)
+                                  .append(", closed = ").append(closed)
+                                  .toString();
+    }
+
+    public ByteBufAllocator getAllocator()
+    {
+        return channelConfig.getAllocator();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/security/SSLFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java
index 3c1293f..a931f5f 100644
--- a/src/java/org/apache/cassandra/security/SSLFactory.java
+++ b/src/java/org/apache/cassandra/security/SSLFactory.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.security;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.net.InetAddress;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.security.KeyStore;
@@ -32,7 +31,6 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLParameters;
 import javax.net.ssl.SSLSocket;
 import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
@@ -79,53 +77,6 @@ public final class SSLFactory
      */
     private static final AtomicReference<SslContext> serverSslContext = new AtomicReference<>();
 
-    /** Create a socket and connect */
-    public static SSLSocket getSocket(EncryptionOptions options, InetAddress address, int port, InetAddress localAddress, int localPort) throws IOException
-    {
-        SSLContext ctx = createSSLContext(options, true);
-        SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket(address, port, localAddress, localPort);
-        try
-        {
-            prepareSocket(socket, options);
-            return socket;
-        }
-        catch (IllegalArgumentException e)
-        {
-            socket.close();
-            throw e;
-        }
-    }
-
-    /** Create a socket and connect, using any local address */
-    public static SSLSocket getSocket(EncryptionOptions options, InetAddress address, int port) throws IOException
-    {
-        SSLContext ctx = createSSLContext(options, true);
-        SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket(address, port);
-        try
-        {
-            prepareSocket(socket, options);
-            return socket;
-        }
-        catch (IllegalArgumentException e)
-        {
-            socket.close();
-            throw e;
-        }
-    }
-
-    /** Sets relevant socket options specified in encryption settings */
-    private static void prepareSocket(SSLSocket socket, EncryptionOptions options)
-    {
-        String[] suites = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
-        if(options.require_endpoint_verification)
-        {
-            SSLParameters sslParameters = socket.getSSLParameters();
-            sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
-            socket.setSSLParameters(sslParameters);
-        }
-        socket.setEnabledCipherSuites(suites);
-    }
-
     /**
      * Create a JSSE {@link SSLContext}.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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 af59733..bab161a 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -1313,17 +1313,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return DatabaseDescriptor.getTruncateRpcTimeout();
     }
 
-    public void setStreamingSocketTimeout(int value)
-    {
-        DatabaseDescriptor.setStreamingSocketTimeout(value);
-        logger.info("set streaming socket timeout to {} ms", value);
-    }
-
-    public int getStreamingSocketTimeout()
-    {
-        return DatabaseDescriptor.getStreamingSocketTimeout();
-    }
-
     public void setStreamThroughputMbPerSec(int value)
     {
         DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(value);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 36c43fd..46b7253 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -502,9 +502,6 @@ public interface StorageServiceMBean extends NotificationEmitter
     public void setTruncateRpcTimeout(long value);
     public long getTruncateRpcTimeout();
 
-    public void setStreamingSocketTimeout(int value);
-    public int getStreamingSocketTimeout();
-
     public void setStreamThroughputMbPerSec(int value);
     public int getStreamThroughputMbPerSec();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
deleted file mode 100644
index 5f734c9..0000000
--- a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
+++ /dev/null
@@ -1,428 +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.BufferedOutputStream;
-import java.io.IOException;
-import java.net.Socket;
-import java.net.SocketException;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-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.AtomicReference;
-
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.netty.util.concurrent.FastThreadLocalThread;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
-import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
-import org.apache.cassandra.net.IncomingStreamingConnection;
-import org.apache.cassandra.streaming.messages.StreamInitMessage;
-import org.apache.cassandra.streaming.messages.StreamMessage;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-/**
- * 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 final StreamSession session;
-
-    private IncomingMessageHandler incoming;
-    private OutgoingMessageHandler outgoing;
-    private final boolean isPreview;
-
-    ConnectionHandler(StreamSession session, int incomingSocketTimeout, boolean isPreview)
-    {
-        this.session = session;
-        this.isPreview = isPreview;
-        this.incoming = new IncomingMessageHandler(session, incomingSocketTimeout);
-        this.outgoing = new OutgoingMessageHandler(session);
-    }
-
-    /**
-     * Set up incoming message handler and initiate streaming.
-     *
-     * This method is called once on initiator.
-     *
-     * @throws IOException
-     */
-    @SuppressWarnings("resource")
-    public void initiate() throws IOException
-    {
-        logger.debug("[Stream #{}] Sending stream init for incoming stream", session.planId());
-        Socket incomingSocket = session.createConnection();
-        incoming.start(incomingSocket, StreamMessage.CURRENT_VERSION, true);
-
-        logger.debug("[Stream #{}] Sending stream init for outgoing stream", session.planId());
-        Socket outgoingSocket = session.createConnection();
-        outgoing.start(outgoingSocket, StreamMessage.CURRENT_VERSION, true);
-    }
-
-    /**
-     * Set up outgoing message handler on receiving side.
-     *
-     * @param connection Incoming connection to use for {@link OutgoingMessageHandler}.
-     * @param version Streaming message version
-     * @throws IOException
-     */
-    public void initiateOnReceivingSide(IncomingStreamingConnection connection, boolean isForOutgoing, int version) throws IOException
-    {
-        if (isForOutgoing)
-            outgoing.start(connection, version);
-        else
-            incoming.start(connection, version);
-    }
-
-    public ListenableFuture<?> close()
-    {
-        logger.debug("[Stream #{}] Closing stream connection handler on {}", session.planId(), session.peer);
-
-        ListenableFuture<?> inClosed = closeIncoming();
-        ListenableFuture<?> outClosed = closeOutgoing();
-
-        return Futures.allAsList(inClosed, outClosed);
-    }
-
-    public ListenableFuture<?> closeOutgoing()
-    {
-        return outgoing == null ? Futures.immediateFuture(null) : outgoing.close();
-    }
-
-    public ListenableFuture<?> closeIncoming()
-    {
-        return incoming == null ? Futures.immediateFuture(null) : incoming.close();
-    }
-
-    /**
-     * 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)
-    {
-        if (outgoing.isClosed())
-            throw new RuntimeException("Outgoing stream handler has been closed");
-
-        if (message.type == StreamMessage.Type.FILE && isPreview)
-            throw new RuntimeException("Cannot send file messages for preview streaming sessions");
-
-        outgoing.enqueue(message);
-    }
-
-    /**
-     * @return true if outgoing connection is opened and ready to send messages
-     */
-    public boolean isOutgoingConnected()
-    {
-        return outgoing != null && !outgoing.isClosed();
-    }
-
-    abstract static class MessageHandler implements Runnable
-    {
-        protected final StreamSession session;
-
-        protected int protocolVersion;
-        private final boolean isOutgoingHandler;
-        protected Socket socket;
-
-        private final AtomicReference<SettableFuture<?>> closeFuture = new AtomicReference<>();
-        private IncomingStreamingConnection incomingConnection;
-
-        protected MessageHandler(StreamSession session, boolean isOutgoingHandler)
-        {
-            this.session = session;
-            this.isOutgoingHandler = isOutgoingHandler;
-        }
-
-        protected abstract String name();
-
-        @SuppressWarnings("resource")
-        protected static DataOutputStreamPlus getWriteChannel(Socket socket) throws IOException
-        {
-            WritableByteChannel out = socket.getChannel();
-            // socket channel is null when encrypted(SSL)
-            if (out == null)
-                return new WrappedDataOutputStreamPlus(new BufferedOutputStream(socket.getOutputStream()));
-            return new BufferedDataOutputStreamPlus(out);
-        }
-
-        protected static ReadableByteChannel getReadChannel(Socket socket) throws IOException
-        {
-            //we do this instead of socket.getChannel() so socketSoTimeout is respected
-            return Channels.newChannel(socket.getInputStream());
-        }
-
-        @SuppressWarnings("resource")
-        private void sendInitMessage() throws IOException
-        {
-            StreamInitMessage message = new StreamInitMessage(FBUtilities.getBroadcastAddress(),
-                                                              session.sessionIndex(),
-                                                              session.planId(),
-                                                              session.streamOperation(),
-                                                              !isOutgoingHandler,
-                                                              session.keepSSTableLevel(),
-                                                              session.getPendingRepair(),
-                                                              session.getPreviewKind());
-            ByteBuffer messageBuf = message.createMessage(false, protocolVersion);
-            DataOutputStreamPlus out = getWriteChannel(socket);
-            out.write(messageBuf);
-            out.flush();
-        }
-
-        public void start(IncomingStreamingConnection connection, int protocolVersion) throws IOException
-        {
-            this.incomingConnection = connection;
-            start(connection.socket, protocolVersion, false);
-        }
-
-        public void start(Socket socket, int protocolVersion, boolean initiator) throws IOException
-        {
-            this.socket = socket;
-            this.protocolVersion = protocolVersion;
-            if (initiator)
-                sendInitMessage();
-
-            new FastThreadLocalThread(this, name() + "-" + socket.getRemoteSocketAddress()).start();
-        }
-
-        public ListenableFuture<?> close()
-        {
-            // Assume it wasn't closed. Not a huge deal if we create a future on a race
-            SettableFuture<?> future = SettableFuture.create();
-            return closeFuture.compareAndSet(null, future)
-                 ? future
-                 : closeFuture.get();
-        }
-
-        public boolean isClosed()
-        {
-            return closeFuture.get() != null;
-        }
-
-        protected void signalCloseDone()
-        {
-            if (!isClosed())
-                close();
-
-            closeFuture.get().set(null);
-
-            // We can now close the socket
-            if (incomingConnection != null)
-            {
-                //this will close the underlying socket and remove it
-                //from active MessagingService connections (CASSANDRA-11854)
-                incomingConnection.close();
-            }
-            else
-            {
-                //this is an outgoing connection not registered in the MessagingService
-                //so we can close the socket directly
-                try
-                {
-                    socket.close();
-                }
-                catch (IOException e)
-                {
-                    // Erroring out while closing shouldn't happen but is not really a big deal, so just log
-                    // it at DEBUG and ignore otherwise.
-                    logger.debug("Unexpected error while closing streaming connection", e);
-                }
-            }
-        }
-    }
-
-    /**
-     * Incoming streaming message handler
-     */
-    static class IncomingMessageHandler extends MessageHandler
-    {
-        private final int socketTimeout;
-
-        IncomingMessageHandler(StreamSession session, int socketTimeout)
-        {
-            super(session, false);
-            this.socketTimeout = socketTimeout;
-        }
-
-        @Override
-        public void start(Socket socket, int version, boolean initiator) throws IOException
-        {
-            try
-            {
-                socket.setSoTimeout(socketTimeout);
-            }
-            catch (SocketException e)
-            {
-                logger.warn("Could not set incoming socket timeout to {}", socketTimeout, e);
-            }
-            super.start(socket, version, initiator);
-        }
-
-        protected String name()
-        {
-            return "STREAM-IN";
-        }
-
-        @SuppressWarnings("resource")
-        public void run()
-        {
-            try
-            {
-                ReadableByteChannel in = getReadChannel(socket);
-                while (!isClosed())
-                {
-                    // receive message
-                    StreamMessage message = StreamMessage.deserialize(in, protocolVersion, session);
-                    logger.debug("[Stream #{}] Received {}", session.planId(), message);
-                    // Might be null if there is an error during streaming (see FileMessage.deserialize). It's ok
-                    // to ignore here since we'll have asked for a retry.
-                    if (message != null)
-                    {
-                        session.messageReceived(message);
-                    }
-                }
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                session.onError(t);
-            }
-            finally
-            {
-                signalCloseDone();
-            }
-        }
-    }
-
-    /**
-     * 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();
-            }
-        });
-
-        OutgoingMessageHandler(StreamSession session)
-        {
-            super(session, true);
-        }
-
-        protected String name()
-        {
-            return "STREAM-OUT";
-        }
-
-        public void enqueue(StreamMessage message)
-        {
-            messageQueue.put(message);
-        }
-
-        @SuppressWarnings("resource")
-        public void run()
-        {
-            try
-            {
-                DataOutputStreamPlus out = getWriteChannel(socket);
-
-                StreamMessage next;
-                while (!isClosed())
-                {
-                    if ((next = messageQueue.poll(1, TimeUnit.SECONDS)) != null)
-                    {
-                        logger.debug("[Stream #{}] Sending {}", session.planId(), next);
-                        sendMessage(out, next);
-                        if (next.type == StreamMessage.Type.SESSION_FAILED)
-                            close();
-                    }
-                }
-
-                // Sends the last messages on the queue
-                while ((next = messageQueue.poll()) != null)
-                    sendMessage(out, next);
-            }
-            catch (InterruptedException e)
-            {
-                throw new AssertionError(e);
-            }
-            catch (Throwable e)
-            {
-                session.onError(e);
-            }
-            finally
-            {
-                signalCloseDone();
-            }
-        }
-
-        private void sendMessage(DataOutputStreamPlus out, StreamMessage message)
-        {
-            try
-            {
-                StreamMessage.serialize(message, out, protocolVersion, session);
-                out.flush();
-                message.sent();
-            }
-            catch (SocketException e)
-            {
-                session.onError(e);
-                close();
-            }
-            catch (IOException e)
-            {
-                session.onError(e);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
index d88d63c..d9ed8be 100644
--- a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
+++ b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
@@ -15,83 +15,93 @@
  * 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.InetSocketAddress;
-import java.net.Socket;
-import java.nio.channels.SocketChannel;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
 
+import com.google.common.util.concurrent.Uninterruptibles;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.Config;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.WriteBufferWaterMark;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.security.SSLFactory;
-import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.apache.cassandra.net.async.NettyFactory;
+import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
+import org.apache.cassandra.net.async.OutboundConnectionParams;
 
 public class DefaultConnectionFactory implements StreamConnectionFactory
 {
     private static final Logger logger = LoggerFactory.getLogger(DefaultConnectionFactory.class);
 
+    private static final int DEFAULT_CHANNEL_BUFFER_SIZE = 1 << 22;
+
+    private static final long MAX_WAIT_TIME_NANOS = TimeUnit.SECONDS.toNanos(30);
     private static final int MAX_CONNECT_ATTEMPTS = 3;
 
-    /**
-     * Connect to peer and start exchanging message.
-     * When connect attempt fails, this retries for maximum of MAX_CONNECT_ATTEMPTS times.
-     *
-     * @param peer the peer to connect to.
-     * @return the created socket.
-     *
-     * @throws IOException when connection failed.
-     */
-    public Socket createConnection(InetAddress peer) throws IOException
+    @Override
+    public Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion) throws IOException
     {
-        int attempts = 0;
+        ServerEncryptionOptions encryptionOptions = DatabaseDescriptor.getServerEncryptionOptions();
+
+        if (encryptionOptions.internode_encryption == ServerEncryptionOptions.InternodeEncryption.none)
+            encryptionOptions = null;
+
+        return createConnection(connectionId, protocolVersion, encryptionOptions);
+    }
+
+    protected Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion, @Nullable ServerEncryptionOptions encryptionOptions) throws IOException
+    {
+        // this is the amount of data to allow in memory before netty sets the channel writablility flag to false
+        int channelBufferSize = DEFAULT_CHANNEL_BUFFER_SIZE;
+        WriteBufferWaterMark waterMark = new WriteBufferWaterMark(channelBufferSize >> 2, channelBufferSize);
+
+        int sendBufferSize = DatabaseDescriptor.getInternodeSendBufferSize() > 0
+                             ? DatabaseDescriptor.getInternodeSendBufferSize()
+                             : OutboundConnectionParams.DEFAULT_SEND_BUFFER_SIZE;
+
+        OutboundConnectionParams params = OutboundConnectionParams.builder()
+                                                                  .connectionId(connectionId)
+                                                                  .encryptionOptions(encryptionOptions)
+                                                                  .mode(NettyFactory.Mode.STREAMING)
+                                                                  .protocolVersion(protocolVersion)
+                                                                  .sendBufferSize(sendBufferSize)
+                                                                  .waterMark(waterMark)
+                                                                  .build();
+
+        Bootstrap bootstrap = NettyFactory.instance.createOutboundBootstrap(params);
+
+        int connectionAttemptCount = 0;
+        long now = System.nanoTime();
+        final long end = now + MAX_WAIT_TIME_NANOS;
+        final Channel channel;
         while (true)
         {
-            try
-            {
-                Socket socket = newSocket(peer);
-                socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
-                socket.setKeepAlive(true);
-                return socket;
-            }
-            catch (IOException e)
+            ChannelFuture channelFuture = bootstrap.connect();
+            channelFuture.awaitUninterruptibly(end - now, TimeUnit.MILLISECONDS);
+            if (channelFuture.isSuccess())
             {
-                if (++attempts >= MAX_CONNECT_ATTEMPTS)
-                    throw e;
-
-                long waitms = DatabaseDescriptor.getRpcTimeout() * (long)Math.pow(2, attempts);
-                logger.warn("Failed attempt {} to connect to {}. Retrying in {} ms. ({})", attempts, peer, waitms, e.getMessage());
-                try
-                {
-                    Thread.sleep(waitms);
-                }
-                catch (InterruptedException wtf)
-                {
-                    throw new IOException("interrupted", wtf);
-                }
+                channel = channelFuture.channel();
+                break;
             }
-        }
-    }
 
-    // TODO this is deliberately copied from (the now former) OutboundTcpConnectionPool, for CASSANDRA-8457.
-    // to be replaced in CASSANDRA-12229 (make streaming use 8457)
-    public static Socket newSocket(InetAddress endpoint) throws IOException
-    {
-        // zero means 'bind on any available port.'
-        if (MessagingService.isEncryptedConnection(endpoint))
-        {
-            return SSLFactory.getSocket(DatabaseDescriptor.getServerEncryptionOptions(), endpoint, DatabaseDescriptor.getSSLStoragePort());
-        }
-        else
-        {
-            SocketChannel channel = SocketChannel.open();
-            channel.connect(new InetSocketAddress(endpoint, DatabaseDescriptor.getStoragePort()));
-            return channel.socket();
+            connectionAttemptCount++;
+            now = System.nanoTime();
+            if (connectionAttemptCount == MAX_CONNECT_ATTEMPTS || end - now <= 0)
+                throw new IOException("failed to connect to " + connectionId + " for streaming data", channelFuture.cause());
+
+            long waitms = DatabaseDescriptor.getRpcTimeout() * (long)Math.pow(2, connectionAttemptCount);
+            logger.warn("Failed attempt {} to connect to {}. Retrying in {} ms.", connectionAttemptCount, connectionId, waitms);
+            Uninterruptibles.sleepUninterruptibly(waitms, TimeUnit.MILLISECONDS);
         }
+
+        return channel;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java b/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java
index dd99611..4cfe41e 100644
--- a/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java
+++ b/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java
@@ -15,16 +15,15 @@
  * 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;
 
-/**
- * Interface that creates connection used by streaming.
- */
+import io.netty.channel.Channel;
+import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
+
 public interface StreamConnectionFactory
 {
-    Socket createConnection(InetAddress peer) throws IOException;
+    Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
index 9059f45..bb8c702 100644
--- a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
+++ b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
@@ -37,8 +37,10 @@ public class StreamCoordinator
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamCoordinator.class);
 
-    // Executor strictly for establishing the initial connections. Once we're connected to the other end the rest of the
-    // streaming is handled directly by the ConnectionHandler's incoming and outgoing threads.
+    /**
+     * Executor strictly for establishing the initial connections. Once we're connected to the other end the rest of the
+     * streaming is handled directly by the {@link StreamingMessageSender}'s incoming and outgoing threads.
+     */
     private static final DebuggableThreadPoolExecutor streamExecutor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("StreamConnectionEstablisher",
                                                                                                                             FBUtilities.getAvailableProcessors());
     private final boolean connectSequentially;
@@ -55,8 +57,8 @@ public class StreamCoordinator
                              boolean connectSequentially, UUID pendingRepair, PreviewKind previewKind)
     {
         this.connectionsPerHost = connectionsPerHost;
-        this.factory = factory;
         this.keepSSTableLevel = keepSSTableLevel;
+        this.factory = factory;
         this.connectSequentially = connectSequentially;
         this.pendingRepair = pendingRepair;
         this.previewKind = previewKind;
@@ -163,6 +165,11 @@ public class StreamCoordinator
         return getOrCreateHostData(peer).getOrCreateSessionById(peer, id, connecting);
     }
 
+    public StreamSession getSessionById(InetAddress peer, int id)
+    {
+        return getHostData(peer).getSessionById(id);
+    }
+
     public synchronized void updateProgress(ProgressInfo info)
     {
         getHostData(info.peer).updateProgress(info);
@@ -274,8 +281,8 @@ public class StreamCoordinator
 
     private class HostStreamingData
     {
-        private Map<Integer, StreamSession> streamSessions = new HashMap<>();
-        private Map<Integer, SessionInfo> sessionInfos = new HashMap<>();
+        private final Map<Integer, StreamSession> streamSessions = new HashMap<>();
+        private final Map<Integer, SessionInfo> sessionInfos = new HashMap<>();
 
         private int lastReturned = -1;
 
@@ -333,6 +340,11 @@ public class StreamCoordinator
             return session;
         }
 
+        public StreamSession getSessionById(int id)
+        {
+            return streamSessions.get(id);
+        }
+
         public void updateProgress(ProgressInfo info)
         {
             sessionInfos.get(info.sessionIndex).updateProgress(info);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 52652c0..a44f02e 100644
--- a/src/java/org/apache/cassandra/streaming/StreamManager.java
+++ b/src/java/org/apache/cassandra/streaming/StreamManager.java
@@ -21,7 +21,6 @@ import java.net.InetAddress;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-
 import javax.management.ListenerNotFoundException;
 import javax.management.MBeanNotificationInfo;
 import javax.management.NotificationFilter;
@@ -136,7 +135,7 @@ public class StreamManager implements StreamManagerMBean
         initiatedStreams.put(result.planId, result);
     }
 
-    public void registerReceiving(final StreamResultFuture result)
+    public StreamResultFuture registerReceiving(final StreamResultFuture result)
     {
         result.addEventListener(notifier);
         // Make sure we remove the stream on completion (whether successful or not)
@@ -148,7 +147,8 @@ public class StreamManager implements StreamManagerMBean
             }
         }, MoreExecutors.directExecutor());
 
-        receivingStreams.put(result.planId, result);
+        StreamResultFuture previous = receivingStreams.putIfAbsent(result.planId, result);
+        return previous ==  null ? result : previous;
     }
 
     public StreamResultFuture getReceivingStream(UUID planId)
@@ -175,4 +175,22 @@ public class StreamManager implements StreamManagerMBean
     {
         return notifier.getNotificationInfo();
     }
+
+    public StreamSession findSession(InetAddress peer, UUID planId, int sessionIndex)
+    {
+        StreamSession session = findSession(initiatedStreams, peer, planId, sessionIndex);
+        if (session !=  null)
+            return session;
+
+        return findSession(receivingStreams, peer, planId, sessionIndex);
+    }
+
+    private StreamSession findSession(Map<UUID, StreamResultFuture> streams, InetAddress peer, UUID planId, int sessionIndex)
+    {
+        StreamResultFuture streamResultFuture = streams.get(planId);
+        if (streamResultFuture == null)
+            return null;
+
+        return streamResultFuture.getSession(peer, sessionIndex);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 05a8d30..213f74b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamPlan.java
+++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java
@@ -33,7 +33,7 @@ import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR
  */
 public class StreamPlan
 {
-    public static final String[] EMPTY_COLUMN_FAMILIES = new String[0];
+    private static final String[] EMPTY_COLUMN_FAMILIES = new String[0];
     private final UUID planId = UUIDGen.getTimeUUID();
     private final StreamOperation streamOperation;
     private final List<StreamEventHandler> handlers = new ArrayList<>();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org

[02/11] cassandra git commit: move streaming to use netty

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java
new file mode 100644
index 0000000..cc6f9e0
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java
@@ -0,0 +1,268 @@
+/*
+ * 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.async;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import javax.annotation.Nullable;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.util.ReferenceCountUtil;
+import io.netty.util.concurrent.FastThreadLocalThread;
+import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus;
+import org.apache.cassandra.streaming.StreamManager;
+import org.apache.cassandra.streaming.StreamReceiveException;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.messages.FileMessageHeader;
+import org.apache.cassandra.streaming.messages.IncomingFileMessage;
+import org.apache.cassandra.streaming.messages.KeepAliveMessage;
+import org.apache.cassandra.streaming.messages.StreamInitMessage;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.streaming.async.NettyStreamingMessageSender.createLogTag;
+
+/**
+ * Handles the inbound side of streaming messages and sstable data. From the incoming data, we derserialize the message
+ * and potentially reify partitions and rows and write those out to new sstable files. Because deserialization is a blocking affair,
+ * we can't block the netty event loop. Thus we have a background thread perform all the blocking deserialization.
+ */
+public class StreamingInboundHandler extends ChannelInboundHandlerAdapter
+{
+    private static final Logger logger = LoggerFactory.getLogger(StreamingInboundHandler.class);
+    static final Function<SessionIdentifier, StreamSession> DEFAULT_SESSION_PROVIDER = sid -> StreamManager.instance.findSession(sid.from, sid.planId, sid.sessionIndex);
+
+    private static final int AUTO_READ_LOW_WATER_MARK = 1 << 15;
+    private static final int AUTO_READ_HIGH_WATER_MARK = 1 << 16;
+
+    private final InetSocketAddress remoteAddress;
+    private final int protocolVersion;
+
+    private final StreamSession session;
+
+    /**
+     * A collection of {@link ByteBuf}s that are yet to be processed. Incoming buffers are first dropped into this
+     * structure, and then consumed.
+     * <p>
+     * For thread safety, this structure's resources are released on the consuming thread
+     * (via {@link RebufferingByteBufDataInputPlus#close()},
+     * but the producing side calls {@link RebufferingByteBufDataInputPlus#markClose()} to notify the input that is should close.
+     */
+    private RebufferingByteBufDataInputPlus buffers;
+
+    private volatile boolean closed;
+
+    public StreamingInboundHandler(InetSocketAddress remoteAddress, int protocolVersion, @Nullable StreamSession session)
+    {
+        this.remoteAddress = remoteAddress;
+        this.protocolVersion = protocolVersion;
+        this.session = session;
+    }
+
+    @Override
+    @SuppressWarnings("resource")
+    public void handlerAdded(ChannelHandlerContext ctx)
+    {
+        buffers = new RebufferingByteBufDataInputPlus(AUTO_READ_LOW_WATER_MARK, AUTO_READ_HIGH_WATER_MARK, ctx.channel().config());
+        Thread blockingIOThread = new FastThreadLocalThread(new StreamDeserializingTask(DEFAULT_SESSION_PROVIDER, session, ctx.channel()),
+                                                            String.format("Stream-Deserializer-%s-%s", remoteAddress.toString(), ctx.channel().id()));
+        blockingIOThread.setDaemon(true);
+        blockingIOThread.start();
+    }
+
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object message)
+    {
+        if (!closed && message instanceof ByteBuf)
+            buffers.append((ByteBuf) message);
+        else
+            ReferenceCountUtil.release(message);
+    }
+
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx)
+    {
+        close();
+        ctx.fireChannelInactive();
+    }
+
+    void close()
+    {
+        closed = true;
+        buffers.markClose();
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+    {
+        if (cause instanceof IOException)
+            logger.trace("connection problem while streaming", cause);
+        else
+            logger.warn("exception occurred while in processing streaming file", cause);
+        close();
+    }
+
+    /**
+     * For testing only!!
+     */
+    void setPendingBuffers(RebufferingByteBufDataInputPlus bufChannel)
+    {
+        this.buffers = bufChannel;
+    }
+
+    /**
+     * The task that performs the actual deserialization.
+     */
+    class StreamDeserializingTask implements Runnable
+    {
+        private final Function<SessionIdentifier, StreamSession> sessionProvider;
+        private final Channel channel;
+
+        @VisibleForTesting
+        StreamSession session;
+
+        StreamDeserializingTask(Function<SessionIdentifier, StreamSession> sessionProvider, StreamSession session, Channel channel)
+        {
+            this.sessionProvider = sessionProvider;
+            this.session = session;
+            this.channel = channel;
+        }
+
+        @Override
+        public void run()
+        {
+            try
+            {
+                while (true)
+                {
+                    // do a check of available bytes and possibly sleep some amount of time (then continue).
+                    // this way we can break out of run() sanely or we end up blocking indefintely in StreamMessage.deserialize()
+                    while (buffers.available() == 0)
+                    {
+                        if (closed)
+                            return;
+
+                        Uninterruptibles.sleepUninterruptibly(400, TimeUnit.MILLISECONDS);
+                    }
+
+                    StreamMessage message = StreamMessage.deserialize(buffers, protocolVersion, null);
+
+                    // keep-alives don't necessarily need to be tied to a session (they could be arrive before or after
+                    // wrt session lifecycle, due to races), just log that we received the message and carry on
+                    if (message instanceof KeepAliveMessage)
+                    {
+                        logger.debug("{} Received {}", createLogTag(session, channel), message);
+                        continue;
+                    }
+
+                    if (session == null)
+                        session = deriveSession(message);
+                    logger.debug("{} Received {}", createLogTag(session, channel), message);
+                    session.messageReceived(message);
+                }
+            }
+            catch (EOFException eof)
+            {
+                // ignore
+            }
+            catch (Throwable t)
+            {
+                JVMStabilityInspector.inspectThrowable(t);
+                if (session != null)
+                {
+                    session.onError(t);
+                }
+                else if (t instanceof StreamReceiveException)
+                {
+                    ((StreamReceiveException)t).session.onError(t);
+                }
+                else
+                {
+                    logger.error("{} stream operation from {} failed", createLogTag(session, channel), remoteAddress, t);
+                }
+            }
+            finally
+            {
+                channel.close();
+                closed = true;
+
+                if (buffers != null)
+                    buffers.close();
+            }
+        }
+
+        StreamSession deriveSession(StreamMessage message) throws IOException
+        {
+            StreamSession streamSession = null;
+            // StreamInitMessage starts a new channel, and IncomingFileMessage potentially, as well.
+            // IncomingFileMessage needs a session to be established a priori, though
+            if (message instanceof StreamInitMessage)
+            {
+                assert session == null : "initiator of stream session received a StreamInitMessage";
+                StreamInitMessage init = (StreamInitMessage) message;
+                StreamResultFuture.initReceivingSide(init.sessionIndex, init.planId, init.streamOperation, init.from, channel, init.keepSSTableLevel, init.pendingRepair, init.previewKind);
+                streamSession = sessionProvider.apply(new SessionIdentifier(init.from, init.planId, init.sessionIndex));
+            }
+            else if (message instanceof IncomingFileMessage)
+            {
+                // TODO: it'd be great to check if the session actually exists before slurping in the entire sstable,
+                // but that's a refactoring for another day
+                FileMessageHeader header = ((IncomingFileMessage) message).header;
+                streamSession = sessionProvider.apply(new SessionIdentifier(header.sender, header.planId, header.sessionIndex));
+            }
+
+            if (streamSession == null)
+                throw new IllegalStateException(createLogTag(null, channel) + " no session found for message " + message);
+
+            streamSession.attach(channel);
+            return streamSession;
+        }
+    }
+
+    /**
+     * A simple struct to wrap the data points required to lookup a {@link StreamSession}
+     */
+    static class SessionIdentifier
+    {
+        final InetAddress from;
+        final UUID planId;
+        final int sessionIndex;
+
+        SessionIdentifier(InetAddress from, UUID planId, int sessionIndex)
+        {
+            this.from = from;
+            this.planId = planId;
+            this.sessionIndex = sessionIndex;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/async/package-info.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/async/package-info.java b/src/java/org/apache/cassandra/streaming/async/package-info.java
new file mode 100644
index 0000000..ecf5115
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/async/package-info.java
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+
+/**
+ * <h1>Non-blocking streaming with netty</h1>
+ * This document describes the implementation details of streaming protocol. A listener for a streaming
+ * session listens on the same socket as internode messaging, and participates in the same handshake protocol
+ * That protocol is described in the package-level documentation for {@link org.apache.cassandra.net.async}, and
+ * thus not here.
+ *
+ * Streaming 2.0 was implemented as CASSANDRA-5286. Streaming 2.0 used (the equivalent of) a single thread and
+ * a single socket to transfer sstables sequentially to a peer (either as part of a repair, bootstrap, and so on).
+ * Part of the motivation for switching to netty and a non-blocking model as to enable file transfers to occur
+ * in parallel for a given session.
+ *
+ * Thus, a more detailed approach is required for stream session management.
+ *
+ * <h2>Session setup and management</h2>
+ *
+ * The full details of the session lifecycle are documented in {@link org.apache.cassandra.streaming.StreamSession}.
+ *
+ *
+ * <h2>File transfer</h2>
+ *
+ * When tranferring whole or subsections of an sstable, only the DATA component is shipped. To that end,
+ * there are three "modes" of an sstable transfer that need to be handled somewhat differently:
+ *
+ * 1) uncompressed sstable - data needs to be read into user space so it can be manipulated: checksum validation,
+ * apply stream compression (see next section), and/or TLS encryption.
+ *
+ * 2) compressed sstable, transferred with SSL/TLS - data needs to be read into user space as that is where the TLS encryption
+ * needs to happen. Netty does not allow the pretense of doing zero-copy transfers when TLS is in the pipeline;
+ * data must explicitly be pulled into user-space memory for TLS encryption to work.
+ *
+ * 3) compressed sstable, transferred without SSL/TLS - data can be streamed via zero-copy transfer as the data does not
+ * need to be manipulated (it can be sent "as-is").
+ *
+ * <h3>Compressing the data</h3>
+ * We always want to transfer as few bytes as possible of the wire when streaming a file. If the
+ * sstable is not already compressed via table compression options, we apply an on-the-fly stream compression
+ * to the data. The stream compression format is documented in
+ * {@link org.apache.cassandra.streaming.async.StreamCompressionSerializer}
+ *
+ * You may be wondering: why implement your own compression scheme? why not use netty's built-in compression codecs,
+ * like {@link io.netty.handler.codec.compression.Lz4FrameEncoder}? That makes complete sense if all the sstables
+ * to be streamed are non using sstable compression (and obviously you wouldn't use stream compression when the sstables
+ * are using sstable compression). The problem is when you have a mix of files, some using sstable compression
+ * and some not. You can either:
+ *
+ * - send the files of one type over one kind of socket, and the others over another socket
+ * - send them both over the same socket, but then auto-adjust per each file type.
+ *
+ * I've opted for the latter to keep socket/channel management simpler and cleaner.
+ */
+package org.apache.cassandra.streaming.async;
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/compress/ByteBufCompressionDataOutputStreamPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/ByteBufCompressionDataOutputStreamPlus.java b/src/java/org/apache/cassandra/streaming/compress/ByteBufCompressionDataOutputStreamPlus.java
new file mode 100644
index 0000000..3f1b22b
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/compress/ByteBufCompressionDataOutputStreamPlus.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.streaming.compress;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4Factory;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
+import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter;
+import org.apache.cassandra.streaming.async.StreamCompressionSerializer;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+
+/**
+ * The intent of this class is to only be used in a very narrow use-case: on the stream compression path of streaming.
+ * This class should really only get calls to {@link #write(ByteBuffer)}, where the incoming buffer is compressed and sent
+ * downstream.
+ */
+public class ByteBufCompressionDataOutputStreamPlus extends WrappedDataOutputStreamPlus
+{
+    private final StreamRateLimiter limiter;
+    private final LZ4Compressor compressor;
+    private final StreamCompressionSerializer serializer;
+
+    public ByteBufCompressionDataOutputStreamPlus(DataOutputStreamPlus out, StreamRateLimiter limiter)
+    {
+        super(out);
+        assert out instanceof ByteBufDataOutputStreamPlus;
+        compressor = LZ4Factory.fastestInstance().fastCompressor();
+        serializer = new StreamCompressionSerializer(((ByteBufDataOutputStreamPlus)out).getAllocator());
+        this.limiter = limiter;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Compress the incoming buffer and send the result downstream. The buffer parameter will not be used nor passed
+     * to downstream components, and thus callers can safely free the buffer upon return.
+     */
+    @Override
+    public void write(ByteBuffer buffer) throws IOException
+    {
+        ByteBuf compressed = serializer.serialize(compressor, buffer, StreamMessage.CURRENT_VERSION);
+
+        // this is a blocking call - you have been warned
+        limiter.acquire(compressed.readableBytes());
+
+        ((ByteBufDataOutputStreamPlus)out).writeToChannel(compressed);
+    }
+
+    @Override
+    public void close()
+    {
+        // explicitly overriding close() to avoid closing the wrapped stream; it will be closed via other means
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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 003db61..76f76ea 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
@@ -19,7 +19,8 @@ package org.apache.cassandra.streaming.compress;
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
 import java.util.Iterator;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
@@ -27,42 +28,45 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.function.Supplier;
 
 import com.google.common.collect.Iterators;
-import com.google.common.primitives.Ints;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import io.netty.util.concurrent.FastThreadLocalThread;
 import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RebufferingInputStream;
+import org.apache.cassandra.streaming.StreamReader.StreamDeserializer;
 import org.apache.cassandra.utils.ChecksumType;
 import org.apache.cassandra.utils.WrappedRunnable;
 
 /**
- * InputStream which reads data from underlining source with given {@link CompressionInfo}.
+ * InputStream which reads data from underlining source with given {@link CompressionInfo}. Uses {@link #buffer} as a buffer
+ * for uncompressed data (which is read by stream consumers - {@link StreamDeserializer} in this case).
  */
-public class CompressedInputStream extends InputStream
+public class CompressedInputStream extends RebufferingInputStream
 {
 
     private static final Logger logger = LoggerFactory.getLogger(CompressedInputStream.class);
 
     private final CompressionInfo info;
     // chunk buffer
-    private final BlockingQueue<byte[]> dataBuffer;
+    private final BlockingQueue<ByteBuffer> dataBuffer;
     private final Supplier<Double> crcCheckChanceSupplier;
 
-    // uncompressed bytes
-    private final byte[] buffer;
+    /**
+     * The base offset of the current {@link #buffer} from the beginning of the stream.
+     */
+    private long bufferOffset = 0;
 
-    // offset from the beginning of the buffer
-    protected long bufferOffset = 0;
-    // current position in stream
+    /**
+     * The current {@link CompressedStreamReader#sections} offset in the stream.
+     */
     private long current = 0;
-    // number of bytes in the buffer that are actually valid
-    protected int validBufferBytes = -1;
 
     private final ChecksumType checksumType;
 
-    // raw checksum bytes
-    private final byte[] checksumBytes = new byte[4];
+    private static final int CHECKSUM_LENGTH = 4;
 
     /**
      * Indicates there was a problem when reading from source stream.
@@ -71,9 +75,9 @@ public class CompressedInputStream extends InputStream
      * with the cause of the error when reading from source stream, so it is
      * thrown to the consumer on subsequent read operation.
      */
-    private static final byte[] POISON_PILL = new byte[0];
+    private static final ByteBuffer POISON_PILL = ByteBuffer.wrap(new byte[0]);
 
-    protected volatile IOException readException = null;
+    private volatile IOException readException = null;
 
     private long totalCompressedBytesRead;
 
@@ -81,11 +85,11 @@ public class CompressedInputStream extends InputStream
      * @param source Input source to read compressed data from
      * @param info Compression info
      */
-    public CompressedInputStream(InputStream source, CompressionInfo info, ChecksumType checksumType, Supplier<Double> crcCheckChanceSupplier)
+    public CompressedInputStream(DataInputPlus source, CompressionInfo info, ChecksumType checksumType, Supplier<Double> crcCheckChanceSupplier)
     {
+        super(ByteBuffer.allocateDirect(info.parameters.chunkLength()));
+        buffer.limit(buffer.position()); // force the buffer to appear "consumed" so that it triggers reBuffer on the first read
         this.info = info;
-        this.buffer = new byte[info.parameters.chunkLength()];
-        // buffer is limited to store up to 1024 chunks
         this.dataBuffer = new ArrayBlockingQueue<>(Math.min(info.chunks.length, 1024));
         this.crcCheckChanceSupplier = crcCheckChanceSupplier;
         this.checksumType = checksumType;
@@ -93,19 +97,50 @@ public class CompressedInputStream extends InputStream
         new FastThreadLocalThread(new Reader(source, info, dataBuffer)).start();
     }
 
-    private void decompressNextChunk() throws IOException
+    /**
+     * Invoked when crossing into the next stream boundary in {@link CompressedStreamReader#sections}.
+     */
+    public void position(long position) throws IOException
     {
         if (readException != null)
             throw readException;
 
+        assert position >= current : "stream can only read forward.";
+        current = position;
+
+        if (current > bufferOffset + buffer.limit())
+            reBuffer(false);
+
+        buffer.position((int)(current - bufferOffset));
+    }
+
+    protected void reBuffer() throws IOException
+    {
+        reBuffer(true);
+    }
+
+    private void reBuffer(boolean updateCurrent) throws IOException
+    {
+        if (readException != null)
+        {
+            FileUtils.clean(buffer);
+            buffer = null;
+            throw readException;
+        }
+
+        // increment the offset into the stream based on the current buffer's read count
+        if (updateCurrent)
+            current += buffer.position();
+
         try
         {
-            byte[] compressedWithCRC = dataBuffer.take();
+            ByteBuffer compressedWithCRC = dataBuffer.take();
             if (compressedWithCRC == POISON_PILL)
             {
                 assert readException != null;
                 throw readException;
             }
+
             decompress(compressedWithCRC);
         }
         catch (InterruptedException e)
@@ -114,74 +149,49 @@ public class CompressedInputStream extends InputStream
         }
     }
 
-    @Override
-    public int read() throws IOException
+    private void decompress(ByteBuffer compressed) throws IOException
     {
-        if (current >= bufferOffset + buffer.length || validBufferBytes == -1)
-            decompressNextChunk();
-
-        assert current >= bufferOffset && current < bufferOffset + validBufferBytes;
-
-        return ((int) buffer[(int) (current++ - bufferOffset)]) & 0xff;
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException
-    {
-        long nextCurrent = current + len;
-
-        if (current >= bufferOffset + buffer.length || validBufferBytes == -1)
-            decompressNextChunk();
+        final int compressedChunkLength = info.parameters.chunkLength();
+        int length = compressed.remaining();
 
-        assert nextCurrent >= bufferOffset;
-
-        int read = 0;
-        while (read < len)
+        // uncompress if the buffer size is less than chunk size. else, if the buffer size is equal to the compressedChunkLength,
+        // we assume the buffer is not compressed. see CASSANDRA-10520
+        final boolean releaseCompressedBuffer;
+        if (length - CHECKSUM_LENGTH < compressedChunkLength)
         {
-            int nextLen = Math.min((len - read), (int)((bufferOffset + validBufferBytes) - current));
-
-            System.arraycopy(buffer, (int)(current - bufferOffset), b, off + read, nextLen);
-            read += nextLen;
-
-            current += nextLen;
-            if (read != len)
-                decompressNextChunk();
+            buffer.clear();
+            compressed.limit(length - CHECKSUM_LENGTH);
+            info.parameters.getSstableCompressor().uncompress(compressed, buffer);
+            buffer.flip();
+            releaseCompressedBuffer = true;
         }
-
-        return len;
-    }
-
-    public void position(long position)
-    {
-        assert position >= current : "stream can only read forward.";
-        current = position;
-    }
-
-    private void decompress(byte[] compressed) throws IOException
-    {
-        // uncompress
-        if (compressed.length - checksumBytes.length < info.parameters.maxCompressedLength())
-            validBufferBytes = info.parameters.getSstableCompressor().uncompress(compressed, 0, compressed.length - checksumBytes.length, buffer, 0);
         else
         {
-            validBufferBytes = compressed.length - checksumBytes.length;
-            System.arraycopy(compressed, 0, buffer, 0, validBufferBytes);
+            FileUtils.clean(buffer);
+            buffer = compressed;
+            buffer.limit(length - CHECKSUM_LENGTH);
+            releaseCompressedBuffer = false;
         }
-        totalCompressedBytesRead += compressed.length;
+        totalCompressedBytesRead += length;
 
         // validate crc randomly
         double crcCheckChance = this.crcCheckChanceSupplier.get();
         if (crcCheckChance > 0d && crcCheckChance > ThreadLocalRandom.current().nextDouble())
         {
-            int checksum = (int) checksumType.of(compressed, 0, compressed.length - checksumBytes.length);
+            ByteBuffer crcBuf = compressed.duplicate();
+            crcBuf.limit(length - CHECKSUM_LENGTH).position(0);
+            int checksum = (int) checksumType.of(crcBuf);
 
-            System.arraycopy(compressed, compressed.length - checksumBytes.length, checksumBytes, 0, checksumBytes.length);
-            if (Ints.fromByteArray(checksumBytes) != checksum)
+            crcBuf.limit(length);
+            if (crcBuf.getInt() != checksum)
                 throw new IOException("CRC unmatched");
         }
 
+        if (releaseCompressedBuffer)
+            FileUtils.clean(compressed);
+
         // buffer offset is always aligned
-        bufferOffset = current & ~(buffer.length - 1);
+        bufferOffset = current & ~(compressedChunkLength - 1);
     }
 
     public long getTotalCompressedBytesRead()
@@ -189,13 +199,26 @@ public class CompressedInputStream extends InputStream
         return totalCompressedBytesRead;
     }
 
+    /**
+     * Releases the resources specific to this instance, but not the {@link DataInputPlus} that is used by the {@link Reader}.
+     */
+    @Override
+    public void close()
+    {
+        if (buffer != null)
+        {
+            FileUtils.clean(buffer);
+            buffer = null;
+        }
+    }
+
     class Reader extends WrappedRunnable
     {
-        private final InputStream source;
+        private final DataInputPlus source;
         private final Iterator<CompressionMetadata.Chunk> chunks;
-        private final BlockingQueue<byte[]> dataBuffer;
+        private final BlockingQueue<ByteBuffer> dataBuffer;
 
-        Reader(InputStream source, CompressionInfo info, BlockingQueue<byte[]> dataBuffer)
+        Reader(DataInputPlus source, CompressionInfo info, BlockingQueue<ByteBuffer> dataBuffer)
         {
             this.source = source;
             this.chunks = Iterators.forArray(info.chunks);
@@ -204,36 +227,54 @@ public class CompressedInputStream extends InputStream
 
         protected void runMayThrow() throws Exception
         {
-            byte[] compressedWithCRC;
+            byte[] tmp = null;
             while (chunks.hasNext())
             {
                 CompressionMetadata.Chunk chunk = chunks.next();
 
                 int readLength = chunk.length + 4; // read with CRC
-                compressedWithCRC = new byte[readLength];
-
-                int bufferRead = 0;
-                while (bufferRead < readLength)
+                ByteBuffer compressedWithCRC = null;
+                try
                 {
-                    try
+                    final int r;
+                    if (source instanceof ReadableByteChannel)
                     {
-                        int r = source.read(compressedWithCRC, bufferRead, readLength - bufferRead);
-                        if (r < 0)
-                        {
-                            readException = new EOFException("No chunk available");
-                            dataBuffer.put(POISON_PILL);
-                            return; // throw exception where we consume dataBuffer
-                        }
-                        bufferRead += r;
+                        compressedWithCRC = ByteBuffer.allocateDirect(readLength);
+                        r = ((ReadableByteChannel)source).read(compressedWithCRC);
+                        compressedWithCRC.flip();
                     }
-                    catch (IOException e)
+                    else
                     {
-                        logger.warn("Error while reading compressed input stream.", e);
-                        readException = e;
+                        // read into an on-heap araay, then copy over to an off-heap buffer. at a minumum snappy requires
+                        // off-heap buffers for decompression, else we could have just wrapped the plain byte array in a ByteBuffer
+                        if (tmp == null || tmp.length < info.parameters.chunkLength() + CHECKSUM_LENGTH)
+                            tmp = new byte[info.parameters.chunkLength() + CHECKSUM_LENGTH];
+                        source.readFully(tmp, 0, readLength);
+                        compressedWithCRC = ByteBuffer.allocateDirect(readLength);
+                        compressedWithCRC.put(tmp, 0, readLength);
+                        compressedWithCRC.position(0);
+                        r = readLength;
+                    }
+
+                    if (r < 0)
+                    {
+                        FileUtils.clean(compressedWithCRC);
+                        readException = new EOFException("No chunk available");
                         dataBuffer.put(POISON_PILL);
                         return; // throw exception where we consume dataBuffer
                     }
                 }
+                catch (IOException e)
+                {
+                    if (!(e instanceof EOFException))
+                        logger.warn("Error while reading compressed input stream.", e);
+                    if (compressedWithCRC != null)
+                        FileUtils.clean(compressedWithCRC);
+
+                    readException = e;
+                    dataBuffer.put(POISON_PILL);
+                    return; // throw exception where we consume dataBuffer
+                }
                 dataBuffer.put(compressedWithCRC);
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index f8e4b40..e40788b 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -18,8 +18,6 @@
 package org.apache.cassandra.streaming.compress;
 
 import java.io.IOException;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
 
 import com.google.common.base.Throwables;
 import org.slf4j.Logger;
@@ -28,7 +26,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
-import org.apache.cassandra.io.util.TrackedInputStream;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.TrackedDataInputPlus;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamReader;
 import org.apache.cassandra.streaming.StreamSession;
@@ -59,8 +58,8 @@ public class CompressedStreamReader extends StreamReader
      * @throws java.io.IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
      */
     @Override
-    @SuppressWarnings("resource") // channel needs to remain open, streams on top of it can't be closed
-    public SSTableMultiWriter read(ReadableByteChannel channel) throws IOException
+    @SuppressWarnings("resource") // input needs to remain open, streams on top of it can't be closed
+    public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException
     {
         long totalSize = totalSize();
 
@@ -76,9 +75,9 @@ public class CompressedStreamReader extends StreamReader
                      session.planId(), fileSeqNum, session.peer, repairedAt, totalSize, cfs.keyspace.getName(), pendingRepair,
                      cfs.getTableName());
 
-        CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo,
+        CompressedInputStream cis = new CompressedInputStream(inputPlus, compressionInfo,
                                                               ChecksumType.CRC32, cfs::getCrcCheckChance);
-        TrackedInputStream in = new TrackedInputStream(cis);
+        TrackedDataInputPlus in = new TrackedDataInputPlus(cis);
 
         StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata(), in, inputVersion, getHeader(cfs.metadata()));
         SSTableMultiWriter writer = null;
@@ -120,6 +119,10 @@ public class CompressedStreamReader extends StreamReader
                 throw e;
             throw Throwables.propagate(e);
         }
+        finally
+        {
+            cis.close();
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 185ab22..0e78b7d 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.streaming.compress;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -30,6 +31,8 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.StreamWriter;
@@ -41,7 +44,7 @@ import org.apache.cassandra.utils.Pair;
  */
 public class CompressedStreamWriter extends StreamWriter
 {
-    public static final int CHUNK_SIZE = 10 * 1024 * 1024;
+    private static final int CHUNK_SIZE = 1 << 16;
 
     private static final Logger logger = LoggerFactory.getLogger(CompressedStreamWriter.class);
 
@@ -56,6 +59,8 @@ public class CompressedStreamWriter extends StreamWriter
     @Override
     public void write(DataOutputStreamPlus out) throws IOException
     {
+        assert out instanceof ByteBufDataOutputStreamPlus;
+        ByteBufDataOutputStreamPlus output = (ByteBufDataOutputStreamPlus)out;
         long totalSize = totalSize();
         logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(),
                      sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize);
@@ -79,10 +84,24 @@ public class CompressedStreamWriter extends StreamWriter
                 long bytesTransferred = 0;
                 while (bytesTransferred < length)
                 {
-                    final long bytesTransferredFinal = bytesTransferred;
                     final int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
                     limiter.acquire(toTransfer);
-                    long lastWrite = out.applyToChannel((wbc) -> fc.transferTo(section.left + bytesTransferredFinal, toTransfer, wbc));
+
+                    ByteBuffer outBuffer = ByteBuffer.allocateDirect(toTransfer);
+                    long lastWrite;
+                    try
+                    {
+                        lastWrite = fc.read(outBuffer, section.left + bytesTransferred);
+                        assert lastWrite == toTransfer : String.format("could not read required number of bytes from file to be streamed: read %d bytes, wanted %d bytes", lastWrite, toTransfer);
+                        outBuffer.flip();
+                        output.writeToChannel(outBuffer);
+                    }
+                    catch (IOException e)
+                    {
+                        FileUtils.clean(outBuffer);
+                        throw e;
+                    }
+
                     bytesTransferred += lastWrite;
                     progress += lastWrite;
                     session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/compress/StreamCompressionInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/StreamCompressionInputStream.java b/src/java/org/apache/cassandra/streaming/compress/StreamCompressionInputStream.java
new file mode 100644
index 0000000..4b1459d
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/compress/StreamCompressionInputStream.java
@@ -0,0 +1,78 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.lz4.LZ4FastDecompressor;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.RebufferingInputStream;
+import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus;
+import org.apache.cassandra.streaming.async.StreamCompressionSerializer;
+
+public class StreamCompressionInputStream extends RebufferingInputStream
+{
+    /**
+     * The stream which contains buffers of compressed data that came from the peer.
+     */
+    private final DataInputPlus dataInputPlus;
+
+    private final LZ4FastDecompressor decompressor;
+    private final int protocolVersion;
+    private final StreamCompressionSerializer deserializer;
+
+    /**
+     * The parent, or owning, buffer of the current buffer being read from ({@link super#buffer}).
+     */
+    private ByteBuf currentBuf;
+
+    public StreamCompressionInputStream(DataInputPlus dataInputPlus, int protocolVersion)
+    {
+        super(Unpooled.EMPTY_BUFFER.nioBuffer());
+        currentBuf = Unpooled.EMPTY_BUFFER;
+
+        this.dataInputPlus = dataInputPlus;
+        this.protocolVersion = protocolVersion;
+        this.decompressor = LZ4Factory.fastestInstance().fastDecompressor();
+
+        ByteBufAllocator allocator = dataInputPlus instanceof RebufferingByteBufDataInputPlus
+                                     ? ((RebufferingByteBufDataInputPlus)dataInputPlus).getAllocator()
+                                     : PooledByteBufAllocator.DEFAULT;
+        deserializer = new StreamCompressionSerializer(allocator);
+    }
+
+    @Override
+    public void reBuffer() throws IOException
+    {
+        currentBuf.release();
+        currentBuf = deserializer.deserialize(decompressor, dataInputPlus, protocolVersion);
+        buffer = currentBuf.nioBuffer(0, currentBuf.readableBytes());
+    }
+
+    @Override
+    public void close()
+    {
+        currentBuf.release();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 44ff553..81e16f7 100644
--- a/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
@@ -17,8 +17,7 @@
  */
 package org.apache.cassandra.streaming.messages;
 
-import java.nio.channels.ReadableByteChannel;
-
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
@@ -26,12 +25,17 @@ public class CompleteMessage extends StreamMessage
 {
     public static Serializer<CompleteMessage> serializer = new Serializer<CompleteMessage>()
     {
-        public CompleteMessage deserialize(ReadableByteChannel in, int version, StreamSession session)
+        public CompleteMessage deserialize(DataInputPlus in, int version, StreamSession session)
         {
             return new CompleteMessage();
         }
 
         public void serialize(CompleteMessage message, DataOutputStreamPlus out, int version, StreamSession session) {}
+
+        public long serializedSize(CompleteMessage message, int version)
+        {
+            return 0;
+        }
     };
 
     public CompleteMessage()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index c65e1d4..fedb971 100644
--- a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.streaming.messages;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
@@ -29,7 +30,10 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.compress.CompressionInfo;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDSerializer;
@@ -42,6 +46,8 @@ public class FileMessageHeader
     public static FileMessageHeaderSerializer serializer = new FileMessageHeaderSerializer();
 
     public final TableId tableId;
+    public UUID planId;
+    public int sessionIndex;
     public final int sequenceNumber;
     /** SSTable version */
     public final Version version;
@@ -61,11 +67,15 @@ public class FileMessageHeader
     public final UUID pendingRepair;
     public final int sstableLevel;
     public final SerializationHeader.Component header;
+    public final InetAddress sender;
 
     /* cached size value */
     private transient final long size;
 
-    public FileMessageHeader(TableId tableId,
+    private FileMessageHeader(TableId tableId,
+                             InetAddress sender,
+                             UUID planId,
+                             int sessionIndex,
                              int sequenceNumber,
                              Version version,
                              SSTableFormat.Type format,
@@ -78,6 +88,9 @@ public class FileMessageHeader
                              SerializationHeader.Component header)
     {
         this.tableId = tableId;
+        this.sender = sender;
+        this.planId = planId;
+        this.sessionIndex = sessionIndex;
         this.sequenceNumber = sequenceNumber;
         this.version = version;
         this.format = format;
@@ -93,6 +106,9 @@ public class FileMessageHeader
     }
 
     public FileMessageHeader(TableId tableId,
+                             InetAddress sender,
+                             UUID planId,
+                             int sessionIndex,
                              int sequenceNumber,
                              Version version,
                              SSTableFormat.Type format,
@@ -105,6 +121,9 @@ public class FileMessageHeader
                              SerializationHeader.Component header)
     {
         this.tableId = tableId;
+        this.sender = sender;
+        this.planId = planId;
+        this.sessionIndex = sessionIndex;
         this.sequenceNumber = sequenceNumber;
         this.version = version;
         this.format = format;
@@ -188,11 +207,20 @@ public class FileMessageHeader
         return result;
     }
 
+    public void addSessionInfo(StreamSession session)
+    {
+        planId = session.planId();
+        sessionIndex = session.sessionIndex();
+    }
+
     static class FileMessageHeaderSerializer
     {
         public CompressionInfo serialize(FileMessageHeader header, DataOutputPlus out, int version) throws IOException
         {
             header.tableId.serialize(out);
+            CompactEndpointSerializationHelper.serialize(header.sender, out);
+            UUIDSerializer.serializer.serialize(header.planId, out, version);
+            out.writeInt(header.sessionIndex);
             out.writeInt(header.sequenceNumber);
             out.writeUTF(header.version.toString());
             out.writeUTF(header.format.name);
@@ -224,6 +252,9 @@ public class FileMessageHeader
         public FileMessageHeader deserialize(DataInputPlus in, int version) throws IOException
         {
             TableId tableId = TableId.deserialize(in);
+            InetAddress sender = CompactEndpointSerializationHelper.deserialize(in);
+            UUID planId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+            int sessionIndex = in.readInt();
             int sequenceNumber = in.readInt();
             Version sstableVersion = SSTableFormat.Type.current().info.getVersion(in.readUTF());
             SSTableFormat.Type format = SSTableFormat.Type.validate(in.readUTF());
@@ -239,12 +270,15 @@ public class FileMessageHeader
             int sstableLevel = in.readInt();
             SerializationHeader.Component header =  SerializationHeader.serializer.deserialize(sstableVersion, in);
 
-            return new FileMessageHeader(tableId, sequenceNumber, sstableVersion, format, estimatedKeys, sections, compressionInfo, repairedAt, pendingRepair, sstableLevel, header);
+            return new FileMessageHeader(tableId, sender, planId, sessionIndex, sequenceNumber, sstableVersion, format, estimatedKeys, sections, compressionInfo, repairedAt, pendingRepair, sstableLevel, header);
         }
 
         public long serializedSize(FileMessageHeader header, int version)
         {
             long size = header.tableId.serializedSize();
+            size += CompactEndpointSerializationHelper.serializedSize(header.sender);
+            size += UUIDSerializer.serializer.serializedSize(header.planId, version);
+            size += TypeSizes.sizeof(header.sessionIndex);
             size += TypeSizes.sizeof(header.sequenceNumber);
             size += TypeSizes.sizeof(header.version.toString());
             size += TypeSizes.sizeof(header.format.name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java b/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java
index 03bcaed..9f43982 100644
--- a/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java
@@ -18,15 +18,15 @@
 package org.apache.cassandra.streaming.messages;
 
 import java.io.IOException;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
 
+import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.streaming.StreamReader;
+import org.apache.cassandra.streaming.StreamReceiveException;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.compress.CompressedStreamReader;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -40,21 +40,27 @@ public class IncomingFileMessage extends StreamMessage
     public static Serializer<IncomingFileMessage> serializer = new Serializer<IncomingFileMessage>()
     {
         @SuppressWarnings("resource")
-        public IncomingFileMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+        public IncomingFileMessage deserialize(DataInputPlus input, int version, StreamSession session) throws IOException
         {
-            DataInputPlus input = new DataInputStreamPlus(Channels.newInputStream(in));
             FileMessageHeader header = FileMessageHeader.serializer.deserialize(input, version);
+            session = StreamManager.instance.findSession(header.sender, header.planId, header.sessionIndex);
+            if (session == null)
+                throw new IllegalStateException(String.format("unknown stream session: %s - %d", header.planId, header.sessionIndex));
+            ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(header.tableId);
+            if (cfs == null)
+                throw new StreamReceiveException(session, "CF " + header.tableId + " was dropped during streaming");
+
             StreamReader reader = !header.isCompressed() ? new StreamReader(header, session)
-                    : new CompressedStreamReader(header, session);
+                                                         : new CompressedStreamReader(header, session);
 
             try
             {
-                return new IncomingFileMessage(reader.read(in), header);
+                return new IncomingFileMessage(reader.read(input), header);
             }
             catch (Throwable t)
             {
                 JVMStabilityInspector.inspectThrowable(t);
-                throw t;
+                throw new StreamReceiveException(session, t);
             }
         }
 
@@ -62,6 +68,11 @@ public class IncomingFileMessage extends StreamMessage
         {
             throw new UnsupportedOperationException("Not allowed to call serialize on an incoming file");
         }
+
+        public long serializedSize(IncomingFileMessage message, int version)
+        {
+            throw new UnsupportedOperationException("Not allowed to call serializedSize on an incoming file");
+        }
     };
 
     public FileMessageHeader header;
@@ -77,7 +88,8 @@ public class IncomingFileMessage extends StreamMessage
     @Override
     public String toString()
     {
-        return "File (" + header + ", file: " + sstable.getFilename() + ")";
+        String filename = sstable != null ? sstable.getFilename() : null;
+        return "File (" + header + ", file: " + filename + ")";
     }
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java b/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java
index bfdc72e..f80c617 100644
--- a/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java
@@ -19,8 +19,8 @@
 package org.apache.cassandra.streaming.messages;
 
 import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
 
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
@@ -28,13 +28,18 @@ public class KeepAliveMessage extends StreamMessage
 {
     public static Serializer<KeepAliveMessage> serializer = new Serializer<KeepAliveMessage>()
     {
-        public KeepAliveMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+        public KeepAliveMessage deserialize(DataInputPlus in, int version, StreamSession session) throws IOException
         {
             return new KeepAliveMessage();
         }
 
         public void serialize(KeepAliveMessage message, DataOutputStreamPlus out, int version, StreamSession session)
         {}
+
+        public long serializedSize(KeepAliveMessage message, int version)
+        {
+            return 0;
+        }
     };
 
     public KeepAliveMessage()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
index e3e6b9b..f44b41c 100644
--- a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
@@ -18,17 +18,18 @@
 package org.apache.cassandra.streaming.messages;
 
 import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
 import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.StreamWriter;
 import org.apache.cassandra.streaming.compress.CompressedStreamWriter;
 import org.apache.cassandra.streaming.compress.CompressionInfo;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.Ref;
 
@@ -39,7 +40,7 @@ public class OutgoingFileMessage extends StreamMessage
 {
     public static Serializer<OutgoingFileMessage> serializer = new Serializer<OutgoingFileMessage>()
     {
-        public OutgoingFileMessage deserialize(ReadableByteChannel in, int version, StreamSession session)
+        public OutgoingFileMessage deserialize(DataInputPlus in, int version, StreamSession session)
         {
             throw new UnsupportedOperationException("Not allowed to call deserialize on an outgoing file");
         }
@@ -57,6 +58,11 @@ public class OutgoingFileMessage extends StreamMessage
                 message.finishTransfer();
             }
         }
+
+        public long serializedSize(OutgoingFileMessage message, int version)
+        {
+            return 0;
+        }
     };
 
     public final FileMessageHeader header;
@@ -65,7 +71,7 @@ public class OutgoingFileMessage extends StreamMessage
     private boolean completed = false;
     private boolean transferring = false;
 
-    public OutgoingFileMessage(Ref<SSTableReader> ref, int sequenceNumber, long estimatedKeys, List<Pair<Long, Long>> sections, boolean keepSSTableLevel)
+    public OutgoingFileMessage(Ref<SSTableReader> ref, StreamSession session, int sequenceNumber, long estimatedKeys, List<Pair<Long, Long>> sections, boolean keepSSTableLevel)
     {
         super(Type.FILE);
         this.ref = ref;
@@ -73,6 +79,9 @@ public class OutgoingFileMessage extends StreamMessage
         SSTableReader sstable = ref.get();
         filename = sstable.getFilename();
         this.header = new FileMessageHeader(sstable.metadata().id,
+                                            FBUtilities.getBroadcastAddress(),
+                                            session.planId(),
+                                            session.sessionIndex(),
                                             sequenceNumber,
                                             sstable.descriptor.version,
                                             sstable.descriptor.formatType,
@@ -93,12 +102,12 @@ public class OutgoingFileMessage extends StreamMessage
         }
 
         CompressionInfo compressionInfo = FileMessageHeader.serializer.serialize(header, out, version);
-
+        out.flush();
         final SSTableReader reader = ref.get();
         StreamWriter writer = compressionInfo == null ?
-                                      new StreamWriter(reader, header.sections, session) :
-                                      new CompressedStreamWriter(reader, header.sections,
-                                                                 compressionInfo, session);
+                              new StreamWriter(reader, header.sections, session) :
+                              new CompressedStreamWriter(reader, header.sections,
+                                                         compressionInfo, session);
         writer.write(out);
     }
 
@@ -140,5 +149,10 @@ public class OutgoingFileMessage extends StreamMessage
     {
         return "File (" + header + ", file: " + filename + ")";
     }
+
+    public String getFilename()
+    {
+        return filename;
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/messages/PrepareAckMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareAckMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareAckMessage.java
new file mode 100644
index 0000000..f43ff01
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareAckMessage.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.IOException;
+
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamSession;
+
+public class PrepareAckMessage extends StreamMessage
+{
+    public static Serializer<PrepareAckMessage> serializer = new Serializer<PrepareAckMessage>()
+    {
+        public void serialize(PrepareAckMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        {
+            //nop
+        }
+
+        public PrepareAckMessage deserialize(DataInputPlus in, int version, StreamSession session) throws IOException
+        {
+            return new PrepareAckMessage();
+        }
+
+        public long serializedSize(PrepareAckMessage message, int version)
+        {
+            return 0;
+        }
+    };
+
+    public PrepareAckMessage()
+    {
+        super(Type.PREPARE_ACK);
+    }
+
+    @Override
+    public String toString()
+    {
+        return "Prepare ACK";
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
deleted file mode 100644
index 1f53be7..0000000
--- a/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
+++ /dev/null
@@ -1,93 +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.messages;
-
-import java.io.*;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.util.ArrayList;
-import java.util.Collection;
-
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-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>()
-    {
-        @SuppressWarnings("resource") // Not closing constructed DataInputPlus's as the channel needs to remain open.
-        public PrepareMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
-        {
-            DataInputPlus input = new DataInputStreamPlus(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, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
-        {
-            // requests
-            out.writeInt(message.requests.size());
-            for (StreamRequest request : message.requests)
-                StreamRequest.serializer.serialize(request, out, version);
-            // summaries
-            out.writeInt(message.summaries.size());
-            for (StreamSummary summary : message.summaries)
-                StreamSummary.serializer.serialize(summary, out, 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("Prepare (");
-        sb.append(requests.size()).append(" requests, ");
-        int totalFile = 0;
-        for (StreamSummary summary : summaries)
-            totalFile += summary.files;
-        sb.append(" ").append(totalFile).append(" files");
-        sb.append('}');
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java
new file mode 100644
index 0000000..2d8026c
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java
@@ -0,0 +1,80 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+
+public class PrepareSynAckMessage extends StreamMessage
+{
+    public static Serializer<PrepareSynAckMessage> serializer = new Serializer<PrepareSynAckMessage>()
+    {
+        public void serialize(PrepareSynAckMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        {
+            out.writeInt(message.summaries.size());
+            for (StreamSummary summary : message.summaries)
+                StreamSummary.serializer.serialize(summary, out, version);
+        }
+
+        public PrepareSynAckMessage deserialize(DataInputPlus input, int version, StreamSession session) throws IOException
+        {
+            PrepareSynAckMessage message = new PrepareSynAckMessage();
+            int numSummaries = input.readInt();
+            for (int i = 0; i < numSummaries; i++)
+                message.summaries.add(StreamSummary.serializer.deserialize(input, version));
+            return message;
+        }
+
+        public long serializedSize(PrepareSynAckMessage message, int version)
+        {
+            long size = 4; // count of requests and count of summaries
+            for (StreamSummary summary : message.summaries)
+                size += StreamSummary.serializer.serializedSize(summary, version);
+            return size;
+        }
+    };
+
+    /**
+     * Summaries of streaming out
+     */
+    public final Collection<StreamSummary> summaries = new ArrayList<>();
+
+    public PrepareSynAckMessage()
+    {
+        super(Type.PREPARE_SYNACK);
+    }
+
+    @Override
+    public String toString()
+    {
+        final StringBuilder sb = new StringBuilder("Prepare SYNACK (");
+        int totalFile = 0;
+        for (StreamSummary summary : summaries)
+            totalFile += summary.files;
+        sb.append(" ").append(totalFile).append(" files");
+        sb.append('}');
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
new file mode 100644
index 0000000..6fbaafa
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
@@ -0,0 +1,98 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamRequest;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+
+public class PrepareSynMessage extends StreamMessage
+{
+    public static Serializer<PrepareSynMessage> serializer = new Serializer<PrepareSynMessage>()
+    {
+        public PrepareSynMessage deserialize(DataInputPlus input, int version, StreamSession session) throws IOException
+        {
+            PrepareSynMessage message = new PrepareSynMessage();
+            // 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 long serializedSize(PrepareSynMessage message, int version)
+        {
+            long size = 4 + 4; // count of requests and count of summaries
+            for (StreamRequest request : message.requests)
+                size += StreamRequest.serializer.serializedSize(request, version);
+            for (StreamSummary summary : message.summaries)
+                size += StreamSummary.serializer.serializedSize(summary, version);
+            return size;
+        }
+
+        public void serialize(PrepareSynMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        {
+            // requests
+            out.writeInt(message.requests.size());
+            for (StreamRequest request : message.requests)
+                StreamRequest.serializer.serialize(request, out, version);
+            // summaries
+            out.writeInt(message.summaries.size());
+            for (StreamSummary summary : message.summaries)
+                StreamSummary.serializer.serialize(summary, out, version);
+        }
+    };
+
+    /**
+     * Streaming requests
+     */
+    public final Collection<StreamRequest> requests = new ArrayList<>();
+
+    /**
+     * Summaries of streaming out
+     */
+    public final Collection<StreamSummary> summaries = new ArrayList<>();
+
+    public PrepareSynMessage()
+    {
+        super(Type.PREPARE_SYN);
+    }
+
+    @Override
+    public String toString()
+    {
+        final StringBuilder sb = new StringBuilder("Prepare SYN (");
+        sb.append(requests.size()).append(" requests, ");
+        int totalFile = 0;
+        for (StreamSummary summary : summaries)
+            totalFile += summary.files;
+        sb.append(" ").append(totalFile).append(" files");
+        sb.append('}');
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
index 55dd7e6..3988dcc 100644
--- a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
@@ -18,11 +18,8 @@
 package org.apache.cassandra.streaming.messages;
 
 import java.io.*;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.StreamSession;
@@ -32,9 +29,8 @@ public class ReceivedMessage extends StreamMessage
     public static Serializer<ReceivedMessage> serializer = new Serializer<ReceivedMessage>()
     {
         @SuppressWarnings("resource") // Not closing constructed DataInputPlus's as the channel needs to remain open.
-        public ReceivedMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+        public ReceivedMessage deserialize(DataInputPlus input, int version, StreamSession session) throws IOException
         {
-            DataInputPlus input = new DataInputStreamPlus(Channels.newInputStream(in));
             return new ReceivedMessage(TableId.deserialize(input), input.readInt());
         }
 
@@ -43,6 +39,11 @@ public class ReceivedMessage extends StreamMessage
             message.tableId.serialize(out);
             out.writeInt(message.sequenceNumber);
         }
+
+        public long serializedSize(ReceivedMessage message, int version)
+        {
+            return message.tableId.serializedSize() + 4;
+        }
     };
 
     public final TableId tableId;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
deleted file mode 100644
index 047fb06..0000000
--- a/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
+++ /dev/null
@@ -1,71 +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.messages;
-
-import java.io.*;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.util.UUID;
-
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.streaming.StreamSession;
-import org.apache.cassandra.utils.UUIDSerializer;
-
-/**
- * @deprecated retry support removed on CASSANDRA-10992
- */
-@Deprecated
-public class RetryMessage extends StreamMessage
-{
-    public static Serializer<RetryMessage> serializer = new Serializer<RetryMessage>()
-    {
-        @SuppressWarnings("resource") // Not closing constructed DataInputPlus's as the channel needs to remain open.
-        public RetryMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
-        {
-            DataInputPlus input = new DataInputStreamPlus(Channels.newInputStream(in));
-            return new RetryMessage(UUIDSerializer.serializer.deserialize(input, MessagingService.current_version), input.readInt());
-        }
-
-        public void serialize(RetryMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
-        {
-            UUIDSerializer.serializer.serialize(message.cfId, out, MessagingService.current_version);
-            out.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;
-    }
-
-    @Override
-    public String toString()
-    {
-        final StringBuilder sb = new StringBuilder("Retry (");
-        sb.append(cfId).append(", #").append(sequenceNumber).append(')');
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 4a5b6df..59ad90e 100644
--- a/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
@@ -17,8 +17,7 @@
  */
 package org.apache.cassandra.streaming.messages;
 
-import java.nio.channels.ReadableByteChannel;
-
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
@@ -26,12 +25,17 @@ public class SessionFailedMessage extends StreamMessage
 {
     public static Serializer<SessionFailedMessage> serializer = new Serializer<SessionFailedMessage>()
     {
-        public SessionFailedMessage deserialize(ReadableByteChannel in, int version, StreamSession session)
+        public SessionFailedMessage deserialize(DataInputPlus in, int version, StreamSession session)
         {
             return new SessionFailedMessage();
         }
 
         public void serialize(SessionFailedMessage message, DataOutputStreamPlus out, int version, StreamSession session) {}
+
+        public long serializedSize(SessionFailedMessage message, int version)
+        {
+            return 0;
+        }
     };
 
     public SessionFailedMessage()


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org

[09/11] cassandra git commit: switch internode messaging to netty

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/ChannelWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/ChannelWriter.java b/src/java/org/apache/cassandra/net/async/ChannelWriter.java
new file mode 100644
index 0000000..e984736
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/ChannelWriter.java
@@ -0,0 +1,418 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundBuffer;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.MessageSizeEstimator;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.util.Attribute;
+import io.netty.util.AttributeKey;
+import io.netty.util.concurrent.Future;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.CoalescingStrategies;
+import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
+
+/**
+ * Represents a ready and post-handshake channel that can send outbound messages. This class groups a netty channel
+ * with any other channel-related information we track and, most importantly, handles the details on when the channel is flushed.
+ *
+ * <h2>Flushing</h2>
+ *
+ * We don't flush to the socket on every message as it's a bit of a performance drag (making the system call, copying
+ * the buffer, sending out a small packet). Thus, by waiting until we have a decent chunk of data (for some definition
+ * of 'decent'), we can achieve better efficiency and improved performance (yay!).
+ * <p>
+ * When to flush mainly depends on whether we use message coalescing or not (see {@link CoalescingStrategies}).
+ * <p>
+ * Note that the callback functions are invoked on the netty event loop, which is (in almost all cases) different
+ * from the thread that will be invoking {@link #write(QueuedMessage, boolean)}.
+ *
+ * <h3>Flushing without coalescing</h3>
+ *
+ * When no coalescing is in effect, we want to send new message "right away". However, as said above, flushing after
+ * every message would be particularly inefficient when there is lots of message in our sending queue, and so in
+ * practice we want to flush in 2 cases:
+ *  1) After any message <b>if</b> there is no pending message in the send queue.
+ *  2) When we've filled up or exceeded the netty outbound buffer (see {@link ChannelOutboundBuffer})
+ * <p>
+ * The second part is relatively simple and handled generically in {@link MessageOutHandler#write(ChannelHandlerContext, Object, ChannelPromise)} [1].
+ * The first part however is made a little more complicated by how netty's event loop executes. It is woken up by
+ * external callers to the channel invoking a flush, via either {@link Channel#flush} or one of the {@link Channel#writeAndFlush}
+ * methods [2]. So a plain {@link Channel#write} will only queue the message in the channel, and not wake up the event loop.
+ * <p>
+ * This means we don't want to simply call {@link Channel#write} as we want the message processed immediately. But we
+ * also don't want to flush on every message if there is more in the sending queue, so simply calling
+ * {@link Channel#writeAndFlush} isn't completely appropriate either. In practice, we handle this by calling
+ * {@link Channel#writeAndFlush} (so the netty event loop <b>does</b> wake up), but we override the flush behavior so
+ * it actually only flushes if there are no pending messages (see how {@link MessageOutHandler#flush} delegates the flushing
+ * decision back to this class through {@link #onTriggeredFlush}, and how {@link SimpleChannelWriter} makes this a no-op;
+ * instead {@link SimpleChannelWriter} flushes after any message if there are no more pending ones in
+ * {@link #onMessageProcessed}).
+ *
+ * <h3>Flushing with coalescing</h3>
+ *
+ * The goal of coalescing is to (artificially) delay the flushing of data in order to aggregate even more data before
+ * sending a group of packets out. So we don't want to flush after messages even if there is no pending messages in the
+ * sending queue, but we rather want to delegate the decision on when to flush to the {@link CoalescingStrategy}. In
+ * pratice, when coalescing is enabled we will flush in 2 cases:
+ *  1) When the coalescing strategies decides that we should.
+ *  2) When we've filled up or exceeded the netty outbound buffer ({@link ChannelOutboundBuffer}), exactly like in the
+ *  no coalescing case.
+ *  <p>
+ *  The second part is handled exactly like in the no coalescing case, see above.
+ *  The first part is handled by {@link CoalescingChannelWriter#write(QueuedMessage, boolean)}. Whenever a message is sent, we check
+ *  if a flush has been already scheduled by the coalescing strategy. If one has, we're done, otherwise we ask the
+ *  strategy when the next flush should happen and schedule one.
+ *
+ *<h2>Message timeouts and retries</h2>
+ *
+ * The main outward-facing method is {@link #write(QueuedMessage, boolean)}, where callers pass a
+ * {@link QueuedMessage}. If a message times out, as defined in {@link QueuedMessage#isTimedOut()},
+ * the message listener {@link #handleMessageFuture(Future, QueuedMessage, boolean)} is invoked
+ * with the cause being a {@link ExpiredException}. The message is not retried and it is dropped on the floor.
+ * <p>
+ * If there is some {@link IOException} on the socket after the message has been written to the netty channel,
+ * the message listener {@link #handleMessageFuture(Future, QueuedMessage, boolean)} is invoked
+ * and 1) we check to see if the connection should be re-established, and 2) possibly createRetry the message.
+ *
+ * <h2>Failures</h2>
+ *
+ * <h3>Failure to make progress sending bytes</h3>
+ * If we are unable to make progress sending messages, we'll receive a netty notification
+ * ({@link IdleStateEvent}) at {@link MessageOutHandler#userEventTriggered(ChannelHandlerContext, Object)}.
+ * We then want to close the socket/channel, and purge any messages in {@link OutboundMessagingConnection#backlog}
+ * to try to free up memory as quickly as possible. Any messages in the netty pipeline will be marked as fail
+ * (as we close the channel), but {@link MessageOutHandler#userEventTriggered(ChannelHandlerContext, Object)} also
+ * sets a channel attribute, {@link #PURGE_MESSAGES_CHANNEL_ATTR} to true. This is essentially as special flag
+ * that we can look at in the promise handler code ({@link #handleMessageFuture(Future, QueuedMessage, boolean)})
+ * to indicate that any backlog should be thrown away.
+ *
+ * <h2>Notes</h2>
+ * [1] For those desperately interested, and only after you've read the entire class-level doc: You can register a custom
+ * {@link MessageSizeEstimator} with a netty channel. When a message is written to the channel, it will check the
+ * message size, and if the max ({@link ChannelOutboundBuffer}) size will be exceeded, a task to signal the "channel
+ * writability changed" will be executed in the channel. That task, however, will wake up the event loop.
+ * Thus if coalescing is enabled, the event loop will wake up prematurely and process (and possibly flush!) the messages
+ * currently in the queue, thus defeating an aspect of coalescing. Hence, we're not using that feature of netty.
+ * [2]: The netty event loop is also woken up by it's internal timeout on the epoll_wait() system call.
+ */
+abstract class ChannelWriter
+{
+    /**
+     * A netty channel {@link Attribute} to indicate, when a channel is closed, any backlogged messages should be purged,
+     * as well. See the class-level documentation for more information.
+     */
+    static final AttributeKey<Boolean> PURGE_MESSAGES_CHANNEL_ATTR = AttributeKey.newInstance("purgeMessages");
+
+    protected final Channel channel;
+    private volatile boolean closed;
+
+    /** Number of currently pending messages on this channel. */
+    final AtomicLong pendingMessageCount = new AtomicLong(0);
+
+    /**
+     * A consuming function that handles the result of each message sent.
+     */
+    private final Consumer<MessageResult> messageResultConsumer;
+
+    /**
+     * A reusable instance to avoid creating garbage on preciessing the result of every message sent.
+     * As we have the guarantee that the netty evet loop is single threaded, there should be no contention over this
+     * instance, as long as it (not it's state) is shared across threads.
+     */
+    private final MessageResult messageResult = new MessageResult();
+
+    protected ChannelWriter(Channel channel, Consumer<MessageResult> messageResultConsumer)
+    {
+        this.channel = channel;
+        this.messageResultConsumer = messageResultConsumer;
+        channel.attr(PURGE_MESSAGES_CHANNEL_ATTR).set(false);
+    }
+
+    /**
+     * Creates a new {@link ChannelWriter} using the (assumed properly connected) provided channel, and using coalescing
+     * based on the provided strategy.
+     */
+    static ChannelWriter create(Channel channel, Consumer<MessageResult> messageResultConsumer, Optional<CoalescingStrategy> coalescingStrategy)
+    {
+        return coalescingStrategy.isPresent()
+               ? new CoalescingChannelWriter(channel, messageResultConsumer, coalescingStrategy.get())
+               : new SimpleChannelWriter(channel, messageResultConsumer);
+    }
+
+    /**
+     * Writes a message to this {@link ChannelWriter} if the channel is writable.
+     * <p>
+     * We always want to write to the channel *unless* it's not writable yet still open.
+     * If the channel is closed, the promise will be notifed as a fail (due to channel closed),
+     * and let the handler ({@link #handleMessageFuture(Future, QueuedMessage, boolean)})
+     * do the reconnect magic/dance. Thus we simplify when to reconnect by not burdening the (concurrent) callers
+     * of this method, and instead keep it all in the future handler/event loop (which is single threaded).
+     *
+     * @param message the message to write/send.
+     * @param checkWritability a flag to indicate if the status of the channel should be checked before passing
+     * the message on to the {@link #channel}.
+     * @return true if the message was written to the channel; else, false.
+     */
+    boolean write(QueuedMessage message, boolean checkWritability)
+    {
+        if ( (checkWritability && (channel.isWritable()) || !channel.isOpen()) || !checkWritability)
+        {
+            write0(message).addListener(f -> handleMessageFuture(f, message, true));
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Handles the future of sending a particular message on this {@link ChannelWriter}.
+     * <p>
+     * Note: this is called from the netty event loop, so there is no race across multiple execution of this method.
+     */
+    @VisibleForTesting
+    void handleMessageFuture(Future<? super Void> future, QueuedMessage msg, boolean allowReconnect)
+    {
+        messageResult.setAll(this, msg, future, allowReconnect);
+        messageResultConsumer.accept(messageResult);
+        messageResult.clearAll();
+    }
+
+    boolean shouldPurgeBacklog()
+    {
+        if (!channel.attr(PURGE_MESSAGES_CHANNEL_ATTR).get())
+            return false;
+
+        channel.attr(PURGE_MESSAGES_CHANNEL_ATTR).set(false);
+        return true;
+    }
+
+    /**
+     * Writes a backlog of message to this {@link ChannelWriter}. This is mostly equivalent to calling
+     * {@link #write(QueuedMessage, boolean)} for every message of the provided backlog queue, but
+     * it ignores any coalescing, triggering a flush only once after all messages have been sent.
+     *
+     * @param backlog the backlog of message to send.
+     * @return the count of items written to the channel from the queue.
+     */
+    int writeBacklog(Queue<QueuedMessage> backlog, boolean allowReconnect)
+    {
+        int count = 0;
+        while (true)
+        {
+            if (!channel.isWritable())
+                break;
+
+            QueuedMessage msg = backlog.poll();
+            if (msg == null)
+                break;
+
+            pendingMessageCount.incrementAndGet();
+            ChannelFuture future = channel.write(msg);
+            future.addListener(f -> handleMessageFuture(f, msg, allowReconnect));
+            count++;
+        }
+
+        // as this is an infrequent operation, don't bother coordinating with the instance-level flush task
+        if (count > 0)
+            channel.flush();
+
+        return count;
+    }
+
+    void close()
+    {
+        if (closed)
+            return;
+
+        closed = true;
+        channel.close();
+    }
+
+    long pendingMessageCount()
+    {
+        return pendingMessageCount.get();
+    }
+
+    /**
+     * Close the underlying channel but only after having make sure every pending message has been properly sent.
+     */
+    void softClose()
+    {
+        if (closed)
+            return;
+
+        closed = true;
+        channel.writeAndFlush(Unpooled.EMPTY_BUFFER).addListener(ChannelFutureListener.CLOSE);
+    }
+
+    @VisibleForTesting
+    boolean isClosed()
+    {
+        return closed;
+    }
+
+    /**
+     * Write the message to the {@link #channel}.
+     * <p>
+     * Note: this method, in almost all cases, is invoked from an app-level writing thread, not the netty event loop.
+     */
+    protected abstract ChannelFuture write0(QueuedMessage message);
+
+    /**
+     * Invoked after a message has been processed in the pipeline. Should only be used for essential bookkeeping operations.
+     * <p>
+     * Note: this method is invoked on the netty event loop.
+     */
+    abstract void onMessageProcessed(ChannelHandlerContext ctx);
+
+    /**
+     * Invoked when pipeline receives a flush request.
+     * <p>
+     * Note: this method is invoked on the netty event loop.
+     */
+    abstract void onTriggeredFlush(ChannelHandlerContext ctx);
+
+    /**
+     * Handles the non-coalescing flush case.
+     */
+    @VisibleForTesting
+    static class SimpleChannelWriter extends ChannelWriter
+    {
+        private SimpleChannelWriter(Channel channel, Consumer<MessageResult> messageResultConsumer)
+        {
+            super(channel, messageResultConsumer);
+        }
+
+        protected ChannelFuture write0(QueuedMessage message)
+        {
+            pendingMessageCount.incrementAndGet();
+            // We don't truly want to flush on every message but we do want to wake-up the netty event loop for the
+            // channel so the message is processed right away, which is why we use writeAndFlush. This won't actually
+            // flush, though, because onTriggeredFlush, which MessageOutHandler delegates to, does nothing. We will
+            // flush after the message is processed though if there is no pending one due to onMessageProcessed.
+            // See the class javadoc for context and much more details.
+            return channel.writeAndFlush(message);
+        }
+
+        void onMessageProcessed(ChannelHandlerContext ctx)
+        {
+            if (pendingMessageCount.decrementAndGet() == 0)
+                ctx.flush();
+        }
+
+        void onTriggeredFlush(ChannelHandlerContext ctx)
+        {
+            // Don't actually flush on "normal" flush calls to the channel.
+        }
+    }
+
+    /**
+     * Handles the coalescing flush case.
+     */
+    @VisibleForTesting
+    static class CoalescingChannelWriter extends ChannelWriter
+    {
+        private static final int MIN_MESSAGES_FOR_COALESCE = DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages();
+
+        private final CoalescingStrategy strategy;
+        private final int minMessagesForCoalesce;
+
+        @VisibleForTesting
+        final AtomicBoolean scheduledFlush = new AtomicBoolean(false);
+
+        CoalescingChannelWriter(Channel channel, Consumer<MessageResult> messageResultConsumer, CoalescingStrategy strategy)
+        {
+            this (channel, messageResultConsumer, strategy, MIN_MESSAGES_FOR_COALESCE);
+        }
+
+        @VisibleForTesting
+        CoalescingChannelWriter(Channel channel, Consumer<MessageResult> messageResultConsumer, CoalescingStrategy strategy, int minMessagesForCoalesce)
+        {
+            super(channel, messageResultConsumer);
+            this.strategy = strategy;
+            this.minMessagesForCoalesce = minMessagesForCoalesce;
+        }
+
+        protected ChannelFuture write0(QueuedMessage message)
+        {
+            long pendingCount = pendingMessageCount.incrementAndGet();
+            ChannelFuture future = channel.write(message);
+            strategy.newArrival(message);
+
+            // if we lost the race to set the state, simply write to the channel (no flush)
+            if (!scheduledFlush.compareAndSet(false, true))
+                return future;
+
+            long flushDelayNanos;
+            // if we've hit the minimum number of messages for coalescing or we've run out of coalesce time, flush.
+            // note: we check the exact count, instead of greater than or equal to, of message here to prevent a flush task
+            // for each message (if there's messages coming in on multiple threads). There will be, of course, races
+            // with the consumer decrementing the pending counter, but that's still less excessive flushes.
+            if (pendingCount == minMessagesForCoalesce || (flushDelayNanos = strategy.currentCoalescingTimeNanos()) <= 0)
+            {
+                scheduledFlush.set(false);
+                channel.flush();
+            }
+            else
+            {
+                // calling schedule() on the eventLoop will force it to wake up (if not already executing) and schedule the task
+                channel.eventLoop().schedule(() -> {
+                    // NOTE: this executes on the event loop
+                    scheduledFlush.set(false);
+                    // we execute() the flush() as an additional task rather than immediately in-line as there is a
+                    // race condition when this task runs (executing on the event loop) and a thread that writes the channel (top of this method).
+                    // If this task is picked up but before the scheduledFlush falg is flipped, the other thread writes
+                    // and then checks the scheduledFlush (which is still true) and exits.
+                    // This task changes the flag and if it calls flush() in-line, and netty flushs everything immediately (that is, what's been serialized)
+                    // to the transport as we're on the event loop. The other thread's write became a task that executes *after* this task in the netty queue,
+                    // and if there's not a subsequent followup flush scheduled, that write can be orphaned until another write comes in.
+                    channel.eventLoop().execute(channel::flush);
+                }, flushDelayNanos, TimeUnit.NANOSECONDS);
+            }
+            return future;
+        }
+
+        void onMessageProcessed(ChannelHandlerContext ctx)
+        {
+            pendingMessageCount.decrementAndGet();
+        }
+
+        void onTriggeredFlush(ChannelHandlerContext ctx)
+        {
+            // When coalescing, obey the flush calls normally
+            ctx.flush();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/ExpiredException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/ExpiredException.java b/src/java/org/apache/cassandra/net/async/ExpiredException.java
new file mode 100644
index 0000000..191900c
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/ExpiredException.java
@@ -0,0 +1,28 @@
+/*
+ * 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.async;
+
+/**
+ * Thrown when a {@link QueuedMessage} has timed out (has sat in the netty outbound channel for too long).
+ */
+class ExpiredException extends Exception
+{
+    @SuppressWarnings("ThrowableInstanceNeverThrown")
+    static final ExpiredException INSTANCE = new ExpiredException();
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java b/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
new file mode 100644
index 0000000..9b8df80
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
@@ -0,0 +1,304 @@
+/*
+ * 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.async;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.ByteBufInputStream;
+import io.netty.buffer.ByteBufOutputStream;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
+import org.apache.cassandra.net.MessagingService;
+
+/**
+ * Messages for the handshake phase of the internode protocol.
+ * <p>
+ * The handshake's main purpose is to establish a protocol version that both side can talk, as well as exchanging a few connection
+ * options/parameters. The handshake is composed of 3 messages, the first being sent by the initiator of the connection. The other
+ * side then answer with the 2nd message. At that point, if a version mismatch is detected by the connection initiator,
+ * it will simply disconnect and reconnect with a more appropriate version. But if the version is acceptable, the connection
+ * initiator sends the third message of the protocol, after which it considers the connection ready.
+ * <p>
+ * See below for a more precise description of each of those 3 messages.
+ * <p>
+ * Note that this handshake protocol doesn't fully apply to streaming. For streaming, only the first message is sent,
+ * after which the streaming protocol takes over (not documented here)
+ */
+public class HandshakeProtocol
+{
+    /**
+     * The initial message sent when a node creates a new connection to a remote peer. This message contains:
+     *   1) the {@link MessagingService#PROTOCOL_MAGIC} number (4 bytes).
+     *   2) the connection flags (4 bytes), which encodes:
+     *      - the version the initiator thinks should be used for the connection (in practice, either the initiator
+     *        version if it's the first time we connect to that remote since startup, or the last version known for that
+     *        peer otherwise).
+     *      - the "mode" of the connection: whether it is for streaming or for messaging.
+     *      - whether compression should be used or not (if it is, compression is enabled _after_ the last message of the
+     *        handshake has been sent).
+     * <p>
+     * More precisely, connection flags:
+     * <pre>
+     * {@code
+     *                      1 1 1 1 1 1 1 1 1 1 2 2 2 2 2 2 2 2 2 2 3 3
+     *  0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * |U U C M       |                |                               |
+     * |N N M O       |     VERSION    |             unused            |
+     * |U U P D       |                |                               |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * }
+     * </pre>
+     * UNU - unused bits lowest two bits; from a historical note: used to be "serializer type," which was always Binary
+     * CMP - compression enabled bit
+     * MOD - connection mode. If the bit is on, the connection is for streaming; if the bit is off, it is for inter-node messaging.
+     * VERSION - if a streaming connection, indicates the streaming protocol version {@link org.apache.cassandra.streaming.messages.StreamMessage#CURRENT_VERSION};
+     * if a messaging connection, indicates the messaging protocol version the initiator *thinks* should be used.
+     */
+    public static class FirstHandshakeMessage
+    {
+        /** Contains the PROTOCOL_MAGIC (int) and the flags (int). */
+        private static final int LENGTH = 8;
+
+        final int messagingVersion;
+        final NettyFactory.Mode mode;
+        final boolean compressionEnabled;
+
+        public FirstHandshakeMessage(int messagingVersion, NettyFactory.Mode mode, boolean compressionEnabled)
+        {
+            assert messagingVersion > 0;
+            this.messagingVersion = messagingVersion;
+            this.mode = mode;
+            this.compressionEnabled = compressionEnabled;
+        }
+
+        @VisibleForTesting
+        int encodeFlags()
+        {
+            int flags = 0;
+            if (compressionEnabled)
+                flags |= 1 << 2;
+            if (mode == NettyFactory.Mode.STREAMING)
+                flags |= 1 << 3;
+
+            flags |= (messagingVersion << 8);
+            return flags;
+        }
+
+        public ByteBuf encode(ByteBufAllocator allocator)
+        {
+            ByteBuf buffer = allocator.directBuffer(LENGTH, LENGTH);
+            buffer.writerIndex(0);
+            buffer.writeInt(MessagingService.PROTOCOL_MAGIC);
+            buffer.writeInt(encodeFlags());
+            return buffer;
+        }
+
+        static FirstHandshakeMessage maybeDecode(ByteBuf in) throws IOException
+        {
+            if (in.readableBytes() < LENGTH)
+                return null;
+
+            MessagingService.validateMagic(in.readInt());
+            int flags = in.readInt();
+            int version = MessagingService.getBits(flags, 15, 8);
+            NettyFactory.Mode mode = MessagingService.getBits(flags, 3, 1) == 1
+                                     ? NettyFactory.Mode.STREAMING
+                                     : NettyFactory.Mode.MESSAGING;
+            boolean compressed = MessagingService.getBits(flags, 2, 1) == 1;
+            return new FirstHandshakeMessage(version, mode, compressed);
+        }
+
+        @Override
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof FirstHandshakeMessage))
+                return false;
+
+            FirstHandshakeMessage that = (FirstHandshakeMessage)other;
+            return this.messagingVersion == that.messagingVersion
+                   && this.mode == that.mode
+                   && this.compressionEnabled == that.compressionEnabled;
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(messagingVersion, mode, compressionEnabled);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("FirstHandshakeMessage - messaging version: %d, mode: %s, compress: %b", messagingVersion, mode, compressionEnabled);
+        }
+    }
+
+    /**
+     * The second message of the handshake, sent by the node receiving the {@link FirstHandshakeMessage} back to the
+     * connection initiator. This message contains the messaging version of the peer sending this message,
+     * so {@link org.apache.cassandra.net.MessagingService#current_version}.
+     */
+    static class SecondHandshakeMessage
+    {
+        /** The messaging version sent by the receiving peer (int). */
+        private static final int LENGTH = 4;
+
+        final int messagingVersion;
+
+        SecondHandshakeMessage(int messagingVersion)
+        {
+            this.messagingVersion = messagingVersion;
+        }
+
+        public ByteBuf encode(ByteBufAllocator allocator)
+        {
+            ByteBuf buffer = allocator.directBuffer(LENGTH, LENGTH);
+            buffer.writerIndex(0);
+            buffer.writeInt(messagingVersion);
+            return buffer;
+        }
+
+        static SecondHandshakeMessage maybeDecode(ByteBuf in)
+        {
+            return in.readableBytes() >= LENGTH ? new SecondHandshakeMessage(in.readInt()) : null;
+        }
+
+        @Override
+        public boolean equals(Object other)
+        {
+            return other instanceof SecondHandshakeMessage
+                   && this.messagingVersion == ((SecondHandshakeMessage) other).messagingVersion;
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Integer.hashCode(messagingVersion);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("SecondHandshakeMessage - messaging version: %d", messagingVersion);
+        }
+    }
+
+    /**
+     * The third message of the handshake, sent by the connection initiator on reception of {@link SecondHandshakeMessage}.
+     * This message contains:
+     *   1) the connection initiator's messaging version (4 bytes) - {@link org.apache.cassandra.net.MessagingService#current_version}.
+     *   2) the connection initiator's broadcast address as encoded by {@link org.apache.cassandra.net.CompactEndpointSerializationHelper}.
+     *      This can be either 5 bytes for an IPv4 address, or 17 bytes for an IPv6 one.
+     * <p>
+     * This message concludes the handshake protocol. After that, the connection will used either for streaming, or to
+     * send messages. If the connection is to be compressed, compression is enabled only after this message is sent/received.
+     */
+    static class ThirdHandshakeMessage
+    {
+        /**
+         * The third message contains the version and IP address of the sending node. Because the IP can be either IPv4 or
+         * IPv6, this can be either 9 (4 for version + 5 for IP) or 21 (4 for version + 17 for IP) bytes. Since we can't know
+         * a priori if the IP address will be v4 or v6, go with the minimum required bytes and hope that if the address is
+         * v6, we'll have the extra 12 bytes in the packet.
+         */
+        private static final int MIN_LENGTH = 9;
+
+        final int messagingVersion;
+        final InetAddress address;
+
+        ThirdHandshakeMessage(int messagingVersion, InetAddress address)
+        {
+            this.messagingVersion = messagingVersion;
+            this.address = address;
+        }
+
+        @SuppressWarnings("resource")
+        public ByteBuf encode(ByteBufAllocator allocator)
+        {
+            int bufLength = Integer.BYTES + CompactEndpointSerializationHelper.serializedSize(address);
+            ByteBuf buffer = allocator.directBuffer(bufLength, bufLength);
+            buffer.writerIndex(0);
+            buffer.writeInt(messagingVersion);
+            try
+            {
+                DataOutput bbos = new ByteBufOutputStream(buffer);
+                CompactEndpointSerializationHelper.serialize(address, bbos);
+                return buffer;
+            }
+            catch (IOException e)
+            {
+                // Shouldn't happen, we're serializing in memory.
+                throw new AssertionError(e);
+            }
+        }
+
+        @SuppressWarnings("resource")
+        static ThirdHandshakeMessage maybeDecode(ByteBuf in)
+        {
+            if (in.readableBytes() < MIN_LENGTH)
+                return null;
+
+            in.markReaderIndex();
+            int version = in.readInt();
+            DataInput inputStream = new ByteBufInputStream(in);
+            try
+            {
+                InetAddress address = CompactEndpointSerializationHelper.deserialize(inputStream);
+                return new ThirdHandshakeMessage(version, address);
+            }
+            catch (IOException e)
+            {
+                // makes the assumption we didn't have enough bytes to deserialize an IPv6 address,
+                // as we only check the MIN_LENGTH of the buf.
+                in.resetReaderIndex();
+                return null;
+            }
+        }
+
+        @Override
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof ThirdHandshakeMessage))
+                return false;
+
+            ThirdHandshakeMessage that = (ThirdHandshakeMessage)other;
+            return this.messagingVersion == that.messagingVersion
+                   && Objects.equals(this.address, that.address);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(messagingVersion, address);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("ThirdHandshakeMessage - messaging version: %d, address = %s", messagingVersion, address);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java b/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
new file mode 100644
index 0000000..5ea03dc
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
@@ -0,0 +1,293 @@
+package org.apache.cassandra.net.async;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import javax.net.ssl.SSLSession;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPipeline;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import io.netty.handler.ssl.SslHandler;
+import org.apache.cassandra.auth.IInternodeAuthenticator;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage;
+import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage;
+import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage;
+
+/**
+ * 'Server'-side component that negotiates the internode handshake when establishing a new connection.
+ * This handler will be the first in the netty channel for each incoming connection (secure socket (TLS) notwithstanding),
+ * and once the handshake is successful, it will configure the proper handlers (mostly {@link MessageInHandler})
+ * and remove itself from the working pipeline.
+ */
+class InboundHandshakeHandler extends ByteToMessageDecoder
+{
+    private static final Logger logger = LoggerFactory.getLogger(NettyFactory.class);
+
+    enum State { START, AWAITING_HANDSHAKE_BEGIN, AWAIT_STREAM_START_RESPONSE, AWAIT_MESSAGING_START_RESPONSE, MESSAGING_HANDSHAKE_COMPLETE, HANDSHAKE_FAIL }
+
+    private State state;
+
+    private final IInternodeAuthenticator authenticator;
+    private boolean hasAuthenticated;
+
+    /**
+     * The peer's declared messaging version.
+     */
+    private int version;
+
+    /**
+     * Does the peer support (or want to use) compressed data?
+     */
+    private boolean compressed;
+
+    /**
+     * A future the essentially places a timeout on how long we'll wait for the peer
+     * to complete the next step of the handshake.
+     */
+    private Future<?> handshakeTimeout;
+
+    InboundHandshakeHandler(IInternodeAuthenticator authenticator)
+    {
+        this.authenticator = authenticator;
+        state = State.START;
+    }
+
+    @Override
+    protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out)
+    {
+        try
+        {
+            if (!hasAuthenticated)
+            {
+                logSecureSocketDetails(ctx);
+                if (!handleAuthenticate(ctx.channel().remoteAddress(), ctx))
+                    return;
+            }
+
+            switch (state)
+            {
+                case START:
+                    state = handleStart(ctx, in);
+                    break;
+                case AWAIT_MESSAGING_START_RESPONSE:
+                    state = handleMessagingStartResponse(ctx, in);
+                    break;
+                case HANDSHAKE_FAIL:
+                    throw new IllegalStateException("channel should be closed after determining the handshake failed with peer: " + ctx.channel().remoteAddress());
+                default:
+                    logger.error("unhandled state: " + state);
+                    state = State.HANDSHAKE_FAIL;
+                    ctx.close();
+            }
+        }
+        catch (Exception e)
+        {
+            logger.error("unexpected error while negotiating internode messaging handshake", e);
+            state = State.HANDSHAKE_FAIL;
+            ctx.close();
+        }
+    }
+
+    /**
+     * Ensure the peer is allowed to connect to this node.
+     */
+    @VisibleForTesting
+    boolean handleAuthenticate(SocketAddress socketAddress, ChannelHandlerContext ctx)
+    {
+        // the only reason addr would not be instanceof InetSocketAddress is in unit testing, when netty's EmbeddedChannel
+        // uses EmbeddedSocketAddress. Normally, we'd do an instanceof for that class name, but it's marked with default visibility,
+        // so we can't reference it outside of it's package (and so it doesn't compile).
+        if (socketAddress instanceof InetSocketAddress)
+        {
+            InetSocketAddress addr = (InetSocketAddress)socketAddress;
+            if (!authenticator.authenticate(addr.getAddress(), addr.getPort()))
+            {
+                if (logger.isTraceEnabled())
+                    logger.trace("Failed to authenticate peer {}", addr);
+                ctx.close();
+                return false;
+            }
+        }
+        else if (!socketAddress.getClass().getSimpleName().equals("EmbeddedSocketAddress"))
+        {
+            ctx.close();
+            return false;
+        }
+        hasAuthenticated = true;
+        return true;
+    }
+
+    /**
+     * If the connection is using SSL/TLS, log some details about it.
+     */
+    private void logSecureSocketDetails(ChannelHandlerContext ctx)
+    {
+        SslHandler sslHandler = ctx.pipeline().get(SslHandler.class);
+        if (sslHandler != null)
+        {
+            SSLSession session = sslHandler.engine().getSession();
+            logger.info("connection from peer {}, protocol = {}, cipher suite = {}",
+                        ctx.channel().remoteAddress(), session.getProtocol(), session.getCipherSuite());
+        }
+    }
+
+    /**
+     * Handles receiving the first message in the internode messaging handshake protocol. If the sender's protocol version
+     * is accepted, we respond with the second message of the handshake protocol.
+     */
+    @VisibleForTesting
+    State handleStart(ChannelHandlerContext ctx, ByteBuf in) throws IOException
+    {
+        FirstHandshakeMessage msg = FirstHandshakeMessage.maybeDecode(in);
+        if (msg == null)
+            return State.START;
+
+        logger.trace("received first handshake message from peer {}, message = {}", ctx.channel().remoteAddress(), msg);
+        version = msg.messagingVersion;
+
+        if (msg.mode == NettyFactory.Mode.STREAMING)
+        {
+            // TODO fill in once streaming is moved to netty
+            ctx.close();
+            return State.AWAIT_STREAM_START_RESPONSE;
+        }
+        else
+        {
+            if (version < MessagingService.VERSION_30)
+            {
+                logger.error("Unable to read obsolete message version {} from {}; The earliest version supported is 3.0.0", version, ctx.channel().remoteAddress());
+                ctx.close();
+                return State.HANDSHAKE_FAIL;
+            }
+
+            logger.trace("Connection version {} from {}", version, ctx.channel().remoteAddress());
+            compressed = msg.compressionEnabled;
+
+            // if this version is < the MS version the other node is trying
+            // to connect with, the other node will disconnect
+            ctx.writeAndFlush(new SecondHandshakeMessage(MessagingService.current_version).encode(ctx.alloc()))
+               .addListener(ChannelFutureListener.CLOSE_ON_FAILURE);
+
+            // outbound side will reconnect to change the version
+            if (version > MessagingService.current_version)
+            {
+                logger.info("peer wants to use a messaging version higher ({}) than what this node supports ({})", version, MessagingService.current_version);
+                ctx.close();
+                return State.HANDSHAKE_FAIL;
+            }
+
+            long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getRpcTimeout());
+            handshakeTimeout = ctx.executor().schedule(() -> failHandshake(ctx), timeout, TimeUnit.MILLISECONDS);
+            return State.AWAIT_MESSAGING_START_RESPONSE;
+        }
+    }
+
+    /**
+     * Handles the third (and last) message in the internode messaging handshake protocol. Grabs the protocol version and
+     * IP addr the peer wants to use.
+     */
+    @VisibleForTesting
+    State handleMessagingStartResponse(ChannelHandlerContext ctx, ByteBuf in) throws IOException
+    {
+        ThirdHandshakeMessage msg = ThirdHandshakeMessage.maybeDecode(in);
+        if (msg == null)
+            return State.AWAIT_MESSAGING_START_RESPONSE;
+
+        logger.trace("received third handshake message from peer {}, message = {}", ctx.channel().remoteAddress(), msg);
+        if (handshakeTimeout != null)
+        {
+            handshakeTimeout.cancel(false);
+            handshakeTimeout = null;
+        }
+
+        int maxVersion = msg.messagingVersion;
+        if (maxVersion > MessagingService.current_version)
+        {
+            logger.error("peer wants to use a messaging version higher ({}) than what this node supports ({})", maxVersion, MessagingService.current_version);
+            ctx.close();
+            return State.HANDSHAKE_FAIL;
+        }
+
+        // record the (true) version of the endpoint
+        InetAddress from = msg.address;
+        MessagingService.instance().setVersion(from, maxVersion);
+        logger.trace("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from));
+
+        setupMessagingPipeline(ctx.pipeline(), from, compressed, version);
+        return State.MESSAGING_HANDSHAKE_COMPLETE;
+    }
+
+    @VisibleForTesting
+    void setupMessagingPipeline(ChannelPipeline pipeline, InetAddress peer, boolean compressed, int messagingVersion)
+    {
+        if (compressed)
+            pipeline.addLast(NettyFactory.INBOUND_COMPRESSOR_HANDLER_NAME, NettyFactory.createLz4Decoder(messagingVersion));
+
+        pipeline.addLast("messageInHandler", new MessageInHandler(peer, messagingVersion));
+        pipeline.remove(this);
+    }
+
+    @VisibleForTesting
+    void failHandshake(ChannelHandlerContext ctx)
+    {
+        // we're not really racing on the handshakeTimeout as we're in the event loop,
+        // but, hey, defensive programming is beautiful thing!
+        if (state == State.MESSAGING_HANDSHAKE_COMPLETE || (handshakeTimeout != null && handshakeTimeout.isCancelled()))
+            return;
+
+        state = State.HANDSHAKE_FAIL;
+        ctx.close();
+
+        if (handshakeTimeout != null)
+        {
+            handshakeTimeout.cancel(false);
+            handshakeTimeout = null;
+        }
+    }
+
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx)
+    {
+        logger.trace("Failed to properly handshake with peer {}. Closing the channel.", ctx.channel().remoteAddress());
+        failHandshake(ctx);
+        ctx.fireChannelInactive();
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+    {
+        logger.error("Failed to properly handshake with peer {}. Closing the channel.", ctx.channel().remoteAddress(), cause);
+        failHandshake(ctx);
+    }
+
+    @VisibleForTesting
+    public State getState()
+    {
+        return state;
+    }
+
+    @VisibleForTesting
+    public void setState(State nextState)
+    {
+        state = nextState;
+    }
+
+    @VisibleForTesting
+    void setHandshakeTimeout(Future<?> timeout)
+    {
+        handshakeTimeout = timeout;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/MessageInHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/MessageInHandler.java b/src/java/org/apache/cassandra/net/async/MessageInHandler.java
new file mode 100644
index 0000000..b400512
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/MessageInHandler.java
@@ -0,0 +1,314 @@
+/*
+ * 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.async;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import org.apache.cassandra.db.monitoring.ApproximateTime;
+import org.apache.cassandra.exceptions.UnknownTableException;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
+import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.MessagingService;
+
+/**
+ * Parses out individual messages from the incoming buffers. Each message, both header and payload, is incrementally built up
+ * from the available input data, then passed to the {@link #messageConsumer}.
+ *
+ * Note: this class derives from {@link ByteToMessageDecoder} to take advantage of the {@link ByteToMessageDecoder.Cumulator}
+ * behavior across {@link #decode(ChannelHandlerContext, ByteBuf, List)} invocations. That way we don't have to maintain
+ * the not-fully consumed {@link ByteBuf}s.
+ */
+class MessageInHandler extends ByteToMessageDecoder
+{
+    public static final Logger logger = LoggerFactory.getLogger(MessageInHandler.class);
+
+    /**
+     * The default target for consuming deserialized {@link MessageIn}.
+     */
+    static final BiConsumer<MessageIn, Integer> MESSAGING_SERVICE_CONSUMER = (messageIn, id) -> MessagingService.instance().receive(messageIn, id);
+
+    private enum State
+    {
+        READ_FIRST_CHUNK,
+        READ_IP_ADDRESS,
+        READ_SECOND_CHUNK,
+        READ_PARAMETERS_DATA,
+        READ_PAYLOAD_SIZE,
+        READ_PAYLOAD
+    }
+
+    /**
+     * The byte count for magic, msg id, timestamp values.
+     */
+    @VisibleForTesting
+    static final int FIRST_SECTION_BYTE_COUNT = 12;
+
+    /**
+     * The byte count for the verb id and the number of parameters.
+     */
+    private static final int SECOND_SECTION_BYTE_COUNT = 8;
+
+    private final InetAddress peer;
+    private final int messagingVersion;
+
+    /**
+     * Abstracts out depending directly on {@link MessagingService#receive(MessageIn, int)}; this makes tests more sane
+     * as they don't require nor trigger the entire message processing circus.
+     */
+    private final BiConsumer<MessageIn, Integer> messageConsumer;
+
+    private State state;
+    private MessageHeader messageHeader;
+
+    MessageInHandler(InetAddress peer, int messagingVersion)
+    {
+        this (peer, messagingVersion, MESSAGING_SERVICE_CONSUMER);
+    }
+
+    MessageInHandler(InetAddress peer, int messagingVersion, BiConsumer<MessageIn, Integer> messageConsumer)
+    {
+        this.peer = peer;
+        this.messagingVersion = messagingVersion;
+        this.messageConsumer = messageConsumer;
+        state = State.READ_FIRST_CHUNK;
+    }
+
+    /**
+     * For each new message coming in, builds up a {@link MessageHeader} instance incrementally. This method
+     * attempts to deserialize as much header information as it can out of the incoming {@link ByteBuf}, and
+     * maintains a trivial state machine to remember progress across invocations.
+     */
+    @SuppressWarnings("resource")
+    public void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out)
+    {
+        ByteBufDataInputPlus inputPlus = new ByteBufDataInputPlus(in);
+        try
+        {
+            while (true)
+            {
+                // an imperfect optimization around calling in.readableBytes() all the time
+                int readableBytes = in.readableBytes();
+
+                switch (state)
+                {
+                    case READ_FIRST_CHUNK:
+                        if (readableBytes < FIRST_SECTION_BYTE_COUNT)
+                            return;
+                        MessagingService.validateMagic(in.readInt());
+                        messageHeader = new MessageHeader();
+                        messageHeader.messageId = in.readInt();
+                        int messageTimestamp = in.readInt(); // make sure to read the sent timestamp, even if DatabaseDescriptor.hasCrossNodeTimeout() is not enabled
+                        messageHeader.constructionTime = MessageIn.deriveConstructionTime(peer, messageTimestamp, ApproximateTime.currentTimeMillis());
+                        state = State.READ_IP_ADDRESS;
+                        readableBytes -= FIRST_SECTION_BYTE_COUNT;
+                        // fall-through
+                    case READ_IP_ADDRESS:
+                        // unfortunately, this assumes knowledge of how CompactEndpointSerializationHelper serializes data (the first byte is the size).
+                        // first, check that we can actually read the size byte, then check if we can read that number of bytes.
+                        // the "+ 1" is to make sure we have the size byte in addition to the serialized IP addr count of bytes in the buffer.
+                        int serializedAddrSize;
+                        if (readableBytes < 1 || readableBytes < (serializedAddrSize = in.getByte(in.readerIndex()) + 1))
+                            return;
+                        messageHeader.from = CompactEndpointSerializationHelper.deserialize(inputPlus);
+                        state = State.READ_SECOND_CHUNK;
+                        readableBytes -= serializedAddrSize;
+                        // fall-through
+                    case READ_SECOND_CHUNK:
+                        if (readableBytes < SECOND_SECTION_BYTE_COUNT)
+                            return;
+                        messageHeader.verb = MessagingService.Verb.fromId(in.readInt());
+                        int paramCount = in.readInt();
+                        messageHeader.parameterCount = paramCount;
+                        messageHeader.parameters = paramCount == 0 ? Collections.emptyMap() : new HashMap<>();
+                        state = State.READ_PARAMETERS_DATA;
+                        readableBytes -= SECOND_SECTION_BYTE_COUNT;
+                        // fall-through
+                    case READ_PARAMETERS_DATA:
+                        if (messageHeader.parameterCount > 0)
+                        {
+                            if (!readParameters(in, inputPlus, messageHeader.parameterCount, messageHeader.parameters))
+                                return;
+                            readableBytes = in.readableBytes(); // we read an indeterminate number of bytes for the headers, so just ask the buffer again
+                        }
+                        state = State.READ_PAYLOAD_SIZE;
+                        // fall-through
+                    case READ_PAYLOAD_SIZE:
+                        if (readableBytes < 4)
+                            return;
+                        messageHeader.payloadSize = in.readInt();
+                        state = State.READ_PAYLOAD;
+                        readableBytes -= 4;
+                        // fall-through
+                    case READ_PAYLOAD:
+                        if (readableBytes < messageHeader.payloadSize)
+                            return;
+
+                        // TODO consider deserailizing the messge not on the event loop
+                        MessageIn<Object> messageIn = MessageIn.read(inputPlus, messagingVersion,
+                                                                     messageHeader.messageId, messageHeader.constructionTime, messageHeader.from,
+                                                                     messageHeader.payloadSize, messageHeader.verb, messageHeader.parameters);
+
+                        if (messageIn != null)
+                            messageConsumer.accept(messageIn, messageHeader.messageId);
+
+                        state = State.READ_FIRST_CHUNK;
+                        messageHeader = null;
+                        break;
+                    default:
+                        throw new IllegalStateException("unknown/unhandled state: " + state);
+                }
+            }
+        }
+        catch (Exception e)
+        {
+            exceptionCaught(ctx, e);
+        }
+    }
+
+    /**
+     * @return <code>true</code> if all the parameters have been read from the {@link ByteBuf}; else, <code>false</code>.
+     */
+    private boolean readParameters(ByteBuf in, ByteBufDataInputPlus inputPlus, int parameterCount, Map<String, byte[]> parameters) throws IOException
+    {
+        // makes the assumption that map.size() is a constant time function (HashMap.size() is)
+        while (parameters.size() < parameterCount)
+        {
+            if (!canReadNextParam(in))
+                return false;
+
+            String key = DataInputStream.readUTF(inputPlus);
+            byte[] value = new byte[in.readInt()];
+            in.readBytes(value);
+            parameters.put(key, value);
+        }
+
+        return true;
+    }
+
+    /**
+     * Determine if we can read the next parameter from the {@link ByteBuf}. This method will *always* set the {@code in}
+     * readIndex back to where it was when this method was invoked.
+     *
+     * NOTE: this function would be sooo much simpler if we included a parameters length int in the messaging format,
+     * instead of checking the remaining readable bytes for each field as we're parsing it. c'est la vie ...
+     */
+    @VisibleForTesting
+    static boolean canReadNextParam(ByteBuf in)
+    {
+        in.markReaderIndex();
+        // capture the readableBytes value here to avoid all the virtual function calls.
+        // subtract 6 as we know we'll be reading a short and an int (for the utf and value lengths).
+        final int minimumBytesRequired = 6;
+        int readableBytes = in.readableBytes() - minimumBytesRequired;
+        if (readableBytes < 0)
+            return false;
+
+        // this is a tad invasive, but since we know the UTF string is prefaced with a 2-byte length,
+        // read that to make sure we have enough bytes to read the string itself.
+        short strLen = in.readShort();
+        // check if we can read that many bytes for the UTF
+        if (strLen > readableBytes)
+        {
+            in.resetReaderIndex();
+            return false;
+        }
+        in.skipBytes(strLen);
+        readableBytes -= strLen;
+
+        // check if we can read the value length
+        if (readableBytes < 4)
+        {
+            in.resetReaderIndex();
+            return false;
+        }
+        int valueLength = in.readInt();
+        // check if we read that many bytes for the value
+        if (valueLength > readableBytes)
+        {
+            in.resetReaderIndex();
+            return false;
+        }
+
+        in.resetReaderIndex();
+        return true;
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+    {
+        if (cause instanceof EOFException)
+            logger.trace("eof reading from socket; closing", cause);
+        else if (cause instanceof UnknownTableException)
+            logger.warn("Got message from unknown table while reading from socket; closing", cause);
+        else if (cause instanceof IOException)
+            logger.trace("IOException reading from socket; closing", cause);
+        else
+            logger.warn("Unexpected exception caught in inbound channel pipeline from " + ctx.channel().remoteAddress(), cause);
+
+        ctx.close();
+    }
+
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx) throws Exception
+    {
+        logger.debug("received channel closed message for peer {} on local addr {}", ctx.channel().remoteAddress(), ctx.channel().localAddress());
+        ctx.fireChannelInactive();
+    }
+
+    // should ony be used for testing!!!
+    @VisibleForTesting
+    MessageHeader getMessageHeader()
+    {
+        return messageHeader;
+    }
+
+    /**
+     * A simple struct to hold the message header data as it is being built up.
+     */
+    static class MessageHeader
+    {
+        int messageId;
+        long constructionTime;
+        InetAddress from;
+        MessagingService.Verb verb;
+        int payloadSize;
+
+        Map<String, byte[]> parameters = Collections.emptyMap();
+
+        /**
+         * Total number of incoming parameters.
+         */
+        int parameterCount;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/MessageOutHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/MessageOutHandler.java b/src/java/org/apache/cassandra/net/async/MessageOutHandler.java
new file mode 100644
index 0000000..b4ceb92
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/MessageOutHandler.java
@@ -0,0 +1,324 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundBuffer;
+import io.netty.channel.ChannelPromise;
+import io.netty.handler.codec.UnsupportedMessageTypeException;
+import io.netty.handler.timeout.IdleState;
+import io.netty.handler.timeout.IdleStateEvent;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis;
+import org.apache.cassandra.utils.NoSpamLogger;
+import org.apache.cassandra.utils.UUIDGen;
+
+import static org.apache.cassandra.config.Config.PROPERTY_PREFIX;
+
+/**
+ * A Netty {@link ChannelHandler} for serializing outbound messages.
+ * <p>
+ * On top of transforming a {@link QueuedMessage} into bytes, this handler also feeds back progress to the linked
+ * {@link ChannelWriter} so that the latter can take decision on when data should be flushed (with and without coalescing).
+ * See the javadoc on {@link ChannelWriter} for more details about the callbacks as well as message timeouts.
+ *<p>
+ * Note: this class derives from {@link ChannelDuplexHandler} so we can intercept calls to
+ * {@link #userEventTriggered(ChannelHandlerContext, Object)} and {@link #channelWritabilityChanged(ChannelHandlerContext)}.
+ */
+class MessageOutHandler extends ChannelDuplexHandler
+{
+    private static final Logger logger = LoggerFactory.getLogger(MessageOutHandler.class);
+    private static final NoSpamLogger errorLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.SECONDS);
+
+    /**
+     * The default size threshold for deciding when to auto-flush the channel.
+     */
+    private static final int DEFAULT_AUTO_FLUSH_THRESHOLD = 1 << 16;
+
+    // reatining the pre 4.0 property name for backward compatibility.
+    private static final String AUTO_FLUSH_PROPERTY = PROPERTY_PREFIX + "otc_buffer_size";
+    static final int AUTO_FLUSH_THRESHOLD = Integer.getInteger(AUTO_FLUSH_PROPERTY, DEFAULT_AUTO_FLUSH_THRESHOLD);
+
+    /**
+     * The amount of prefix data, in bytes, before the serialized message.
+     */
+    private static final int MESSAGE_PREFIX_SIZE = 12;
+
+    private final OutboundConnectionIdentifier connectionId;
+
+    /**
+     * The version of the messaging protocol we're communicating at.
+     */
+    private final int targetMessagingVersion;
+
+    /**
+     * The minumum size at which we'll automatically flush the channel.
+     */
+    private final int flushSizeThreshold;
+
+    private final ChannelWriter channelWriter;
+
+    private final Supplier<QueuedMessage> backlogSupplier;
+
+    MessageOutHandler(OutboundConnectionIdentifier connectionId, int targetMessagingVersion, ChannelWriter channelWriter, Supplier<QueuedMessage> backlogSupplier)
+    {
+        this (connectionId, targetMessagingVersion, channelWriter, backlogSupplier, AUTO_FLUSH_THRESHOLD);
+    }
+
+    MessageOutHandler(OutboundConnectionIdentifier connectionId, int targetMessagingVersion, ChannelWriter channelWriter, Supplier<QueuedMessage> backlogSupplier, int flushThreshold)
+    {
+        this.connectionId = connectionId;
+        this.targetMessagingVersion = targetMessagingVersion;
+        this.channelWriter = channelWriter;
+        this.flushSizeThreshold = flushThreshold;
+        this.backlogSupplier = backlogSupplier;
+    }
+
+    @Override
+    public void write(ChannelHandlerContext ctx, Object o, ChannelPromise promise)
+    {
+        // this is a temporary fix until https://github.com/netty/netty/pull/6867 is released (probably netty 4.1.13).
+        // TL;DR a closed channel can still process messages in the pipeline that were queued before the close.
+        // the channel handlers are removed from the channel potentially saync from the close operation.
+        if (!ctx.channel().isOpen())
+        {
+            logger.debug("attempting to process a message in the pipeline, but the channel is closed", ctx.channel().id());
+            return;
+        }
+
+        ByteBuf out = null;
+        try
+        {
+            if (!isMessageValid(o, promise))
+                return;
+
+            QueuedMessage msg = (QueuedMessage) o;
+
+            // frame size includes the magic and and other values *before* the actual serialized message.
+            // note: don't even bother to check the compressed size (if compression is enabled for the channel),
+            // cuz if it's this large already, we're probably screwed anyway
+            long currentFrameSize = MESSAGE_PREFIX_SIZE + msg.message.serializedSize(targetMessagingVersion);
+            if (currentFrameSize > Integer.MAX_VALUE || currentFrameSize < 0)
+            {
+                promise.tryFailure(new IllegalStateException(String.format("%s illegal frame size: %d, ignoring message", connectionId, currentFrameSize)));
+                return;
+            }
+
+            out = ctx.alloc().ioBuffer((int)currentFrameSize);
+
+            captureTracingInfo(msg);
+            serializeMessage(msg, out);
+            ctx.write(out, promise);
+
+            // check to see if we should flush based on buffered size
+            ChannelOutboundBuffer outboundBuffer = ctx.channel().unsafe().outboundBuffer();
+            if (outboundBuffer != null && outboundBuffer.totalPendingWriteBytes() >= flushSizeThreshold)
+                ctx.flush();
+        }
+        catch(Exception e)
+        {
+            if (out != null && out.refCnt() > 0)
+                out.release(out.refCnt());
+            exceptionCaught(ctx, e);
+            promise.tryFailure(e);
+        }
+        finally
+        {
+            // Make sure we signal the outChanel even in case of errors.
+            channelWriter.onMessageProcessed(ctx);
+        }
+    }
+
+    /**
+     * Test to see if the message passed in is a {@link QueuedMessage} and if it has timed out or not. If the checks fail,
+     * this method has the side effect of modifying the {@link ChannelPromise}.
+     */
+    boolean isMessageValid(Object o, ChannelPromise promise)
+    {
+        // optimize for the common case
+        if (o instanceof QueuedMessage)
+        {
+            if (!((QueuedMessage)o).isTimedOut())
+            {
+                return true;
+            }
+            else
+            {
+                promise.tryFailure(ExpiredException.INSTANCE);
+            }
+        }
+        else
+        {
+            promise.tryFailure(new UnsupportedMessageTypeException(connectionId +
+                                                                   " msg must be an instance of " + QueuedMessage.class.getSimpleName()));
+        }
+        return false;
+    }
+
+    /**
+     * Record any tracing data, if enabled on this message.
+     */
+    @VisibleForTesting
+    void captureTracingInfo(QueuedMessage msg)
+    {
+        try
+        {
+            byte[] sessionBytes = msg.message.parameters.get(Tracing.TRACE_HEADER);
+            if (sessionBytes != null)
+            {
+                UUID sessionId = UUIDGen.getUUID(ByteBuffer.wrap(sessionBytes));
+                TraceState state = Tracing.instance.get(sessionId);
+                String message = String.format("Sending %s message to %s, size = %d bytes",
+                                               msg.message.verb, connectionId.connectionAddress(),
+                                               msg.message.serializedSize(targetMessagingVersion) + MESSAGE_PREFIX_SIZE);
+                // session may have already finished; see CASSANDRA-5668
+                if (state == null)
+                {
+                    byte[] traceTypeBytes = msg.message.parameters.get(Tracing.TRACE_TYPE);
+                    Tracing.TraceType traceType = traceTypeBytes == null ? Tracing.TraceType.QUERY : Tracing.TraceType.deserialize(traceTypeBytes[0]);
+                    Tracing.instance.trace(ByteBuffer.wrap(sessionBytes), message, traceType.getTTL());
+                }
+                else
+                {
+                    state.trace(message);
+                    if (msg.message.verb == MessagingService.Verb.REQUEST_RESPONSE)
+                        Tracing.instance.doneWithNonLocalSession(state);
+                }
+            }
+        }
+        catch (Exception e)
+        {
+            logger.warn("{} failed to capture the tracing info for an outbound message, ignoring", connectionId, e);
+        }
+    }
+
+    private void serializeMessage(QueuedMessage msg, ByteBuf out) throws IOException
+    {
+        out.writeInt(MessagingService.PROTOCOL_MAGIC);
+        out.writeInt(msg.id);
+
+        // int cast cuts off the high-order half of the timestamp, which we can assume remains
+        // the same between now and when the recipient reconstructs it.
+        out.writeInt((int) NanoTimeToCurrentTimeMillis.convert(msg.timestampNanos));
+        @SuppressWarnings("resource")
+        DataOutputPlus outStream = new ByteBufDataOutputPlus(out);
+        msg.message.serialize(outStream, targetMessagingVersion);
+
+        // next few lines are for debugging ... massively helpful!!
+        // if we allocated too much buffer for this message, we'll log here.
+        // if we allocated to little buffer space, we would have hit an exception when trying to write more bytes to it
+        if (out.isWritable())
+            errorLogger.error("{} reported message size {}, actual message size {}, msg {}",
+                         connectionId, out.capacity(), out.writerIndex(), msg.message);
+    }
+
+    @Override
+    public void flush(ChannelHandlerContext ctx)
+    {
+        channelWriter.onTriggeredFlush(ctx);
+    }
+
+
+    /**
+     * {@inheritDoc}
+     *
+     * When the channel becomes writable (assuming it was previously unwritable), try to eat through any backlogged messages
+     * {@link #backlogSupplier}. As we're on the event loop when this is invoked, no one else can fill up the netty
+     * {@link ChannelOutboundBuffer}, so we should be able to make decent progress chewing through the backlog
+     * (assuming not large messages). Any messages messages written from {@link OutboundMessagingConnection} threads won't
+     * be processed immediately; they'll be queued up as tasks, and once this function return, those messages can begin
+     * to be consumed.
+     * <p>
+     * Note: this is invoked on the netty event loop.
+     */
+    @Override
+    public void channelWritabilityChanged(ChannelHandlerContext ctx)
+    {
+        if (!ctx.channel().isWritable())
+            return;
+
+        // guarantee at least a minimal amount of progress (one messge from the backlog) by using a do-while loop.
+        do
+        {
+            QueuedMessage msg = backlogSupplier.get();
+            if (msg == null || !channelWriter.write(msg, false))
+                break;
+        } while (ctx.channel().isWritable());
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * If we get an {@link IdleStateEvent} for the write path, we want to close the channel as we can't make progress.
+     * That assumes, of course, that there's any outstanding bytes in the channel to write. We don't necesarrily care
+     * about idleness (for example, gossip channels will be idle most of the time), but instead our concern is
+     * the ability to make progress when there's work to be done.
+     * <p>
+     * Note: this is invoked on the netty event loop.
+     */
+    @Override
+    public void userEventTriggered(ChannelHandlerContext ctx, Object evt)
+    {
+        if (evt instanceof IdleStateEvent && ((IdleStateEvent)evt).state() == IdleState.WRITER_IDLE)
+        {
+            ChannelOutboundBuffer cob = ctx.channel().unsafe().outboundBuffer();
+            if (cob != null && cob.totalPendingWriteBytes() > 0)
+            {
+                ctx.channel().attr(ChannelWriter.PURGE_MESSAGES_CHANNEL_ATTR)
+                   .compareAndSet(Boolean.FALSE, Boolean.TRUE);
+                ctx.close();
+            }
+        }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+    {
+        if (cause instanceof IOException)
+            logger.trace("{} io error", connectionId, cause);
+        else
+            logger.warn("{} error", connectionId, cause);
+
+        ctx.close();
+    }
+
+    @Override
+    public void close(ChannelHandlerContext ctx, ChannelPromise promise)
+    {
+        ctx.flush();
+        ctx.close(promise);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/MessageResult.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/MessageResult.java b/src/java/org/apache/cassandra/net/async/MessageResult.java
new file mode 100644
index 0000000..b0dc4dc
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/MessageResult.java
@@ -0,0 +1,51 @@
+/*
+ * 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.async;
+
+import io.netty.util.concurrent.Future;
+
+/**
+ * A simple, reusable struct that holds the unprocessed result of sending a message via netty. This object is intended
+ * to be reusable to avoid creating a bunch of garbage (just for processing the results of sending a message).
+ *
+ * The intended use is to be a member field in a class, like {@link ChannelWriter}, repopulated on each message result,
+ * and then immediately cleared (via {@link #clearAll()}) when done.
+ */
+public class MessageResult
+{
+    ChannelWriter writer;
+    QueuedMessage msg;
+    Future<? super Void> future;
+    boolean allowReconnect;
+
+    void setAll(ChannelWriter writer, QueuedMessage msg, Future<? super Void> future, boolean allowReconnect)
+    {
+        this.writer = writer;
+        this.msg = msg;
+        this.future = future;
+        this.allowReconnect = allowReconnect;
+    }
+
+    void clearAll()
+    {
+        this.writer = null;
+        this.msg = null;
+        this.future = null;
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[08/11] cassandra git commit: switch internode messaging to netty

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/NettyFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/NettyFactory.java b/src/java/org/apache/cassandra/net/async/NettyFactory.java
new file mode 100644
index 0000000..13d8810
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/NettyFactory.java
@@ -0,0 +1,375 @@
+package org.apache.cassandra.net.async;
+
+import java.net.InetSocketAddress;
+import java.util.zip.Checksum;
+
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLParameters;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.ServerChannel;
+import io.netty.channel.epoll.Epoll;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollServerSocketChannel;
+import io.netty.channel.epoll.EpollSocketChannel;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.compression.Lz4FrameDecoder;
+import io.netty.handler.codec.compression.Lz4FrameEncoder;
+import io.netty.handler.logging.LogLevel;
+import io.netty.handler.logging.LoggingHandler;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslHandler;
+import io.netty.util.concurrent.DefaultEventExecutor;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import io.netty.util.concurrent.EventExecutor;
+import io.netty.util.internal.logging.InternalLoggerFactory;
+import io.netty.util.internal.logging.Slf4JLoggerFactory;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.xxhash.XXHashFactory;
+import org.apache.cassandra.auth.IInternodeAuthenticator;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions.InternodeEncryption;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.security.SSLFactory;
+import org.apache.cassandra.service.NativeTransportService;
+import org.apache.cassandra.utils.ChecksumType;
+import org.apache.cassandra.utils.CoalescingStrategies;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * A factory for building Netty {@link Channel}s. Channels here are setup with a pipeline to participate
+ * in the internode protocol handshake, either the inbound or outbound side as per the method invoked.
+ */
+public final class NettyFactory
+{
+    private static final Logger logger = LoggerFactory.getLogger(NettyFactory.class);
+
+    /**
+     * The block size for use with netty's lz4 code.
+     */
+    private static final int COMPRESSION_BLOCK_SIZE = 1 << 16;
+
+    private static final int LZ4_HASH_SEED = 0x9747b28c;
+
+    public enum Mode { MESSAGING, STREAMING }
+
+    private static final String SSL_CHANNEL_HANDLER_NAME = "ssl";
+    static final String INBOUND_COMPRESSOR_HANDLER_NAME = "inboundCompressor";
+    static final String OUTBOUND_COMPRESSOR_HANDLER_NAME = "outboundCompressor";
+    private static final String HANDSHAKE_HANDLER_NAME = "handshakeHandler";
+
+    /** a useful addition for debugging; simply set to true to get more data in your logs */
+    private static final boolean WIRETRACE = false;
+    static
+    {
+        if (WIRETRACE)
+            InternalLoggerFactory.setDefaultFactory(Slf4JLoggerFactory.INSTANCE);
+    }
+
+    private static final boolean DEFAULT_USE_EPOLL = NativeTransportService.useEpoll();
+    static
+    {
+        if (!DEFAULT_USE_EPOLL)
+            logger.warn("epoll not availble {}", Epoll.unavailabilityCause());
+    }
+
+    /**
+     * The size of the receive queue for the outbound channels. As outbound channels do not receive data
+     * (outside of the internode messaging protocol's handshake), this value can be relatively small.
+     */
+    private static final int OUTBOUND_CHANNEL_RECEIVE_BUFFER_SIZE = 1 << 10;
+
+    /**
+     * The size of the send queue for the inbound channels. As inbound channels do not send data
+     * (outside of the internode messaging protocol's handshake), this value can be relatively small.
+     */
+    private static final int INBOUND_CHANNEL_SEND_BUFFER_SIZE = 1 << 10;
+
+    /**
+     * A factory instance that all normal, runtime code should use. Separate instances should only be used for testing.
+     */
+    public static final NettyFactory instance = new NettyFactory(DEFAULT_USE_EPOLL);
+
+    private final boolean useEpoll;
+    private final EventLoopGroup acceptGroup;
+
+    private final EventLoopGroup inboundGroup;
+    private final EventLoopGroup outboundGroup;
+
+    /**
+     * Constructor that allows modifying the {@link NettyFactory#useEpoll} for testing purposes. Otherwise, use the
+     * default {@link #instance}.
+     */
+    @VisibleForTesting
+    NettyFactory(boolean useEpoll)
+    {
+        this.useEpoll = useEpoll;
+        acceptGroup = getEventLoopGroup(useEpoll, determineAcceptGroupSize(DatabaseDescriptor.getServerEncryptionOptions().internode_encryption),
+                                        "MessagingService-NettyAcceptor-Threads", false);
+        inboundGroup = getEventLoopGroup(useEpoll, FBUtilities.getAvailableProcessors(), "MessagingService-NettyInbound-Threads", false);
+        outboundGroup = getEventLoopGroup(useEpoll, FBUtilities.getAvailableProcessors(), "MessagingService-NettyOutbound-Threads", true);
+    }
+
+    /**
+     * Determine the number of accept threads we need, which is based upon the number of listening sockets we will have.
+     * We'll have either 1 or 2 listen sockets, depending on if we use SSL or not in combination with non-SSL. If we have both,
+     * we'll have two sockets, and thus need two threads; else one socket and one thread.
+     *
+     * If the operator has configured multiple IP addresses (both {@link org.apache.cassandra.config.Config#broadcast_address}
+     * and {@link org.apache.cassandra.config.Config#listen_address} are configured), then we listen on another set of sockets
+     * - basically doubling the count. See CASSANDRA-9748 for more details.
+     */
+    static int determineAcceptGroupSize(InternodeEncryption internode_encryption)
+    {
+        int listenSocketCount = internode_encryption == InternodeEncryption.dc || internode_encryption == InternodeEncryption.rack ? 2 : 1;
+
+        if (MessagingService.shouldListenOnBroadcastAddress())
+            listenSocketCount *= 2;
+
+        return listenSocketCount;
+    }
+
+    /**
+     * Create an {@link EventLoopGroup}, for epoll or nio. The {@code boostIoRatio} flag passes a hint to the netty
+     * event loop threads to optimize comsuming all the tasks from the netty channel before checking for IO activity.
+     * By default, netty will process some maximum number of tasks off it's queue before it will check for activity on
+     * any of the open FDs, which basically amounts to checking for any incoming data. If you have a class of event loops
+     * that that do almost *no* inbound activity (like cassandra's outbound channels), then it behooves us to have the
+     * outbound netty channel consume as many tasks as it can before making the system calls to check up on the FDs,
+     * as we're not expecting any incoming data on those sockets, anyways. Thus, we pass the magic value {@code 100}
+     * to achieve the maximum consuption from the netty queue. (for implementation details, as of netty 4.1.8,
+     * see {@link io.netty.channel.epoll.EpollEventLoop#run()}.
+     */
+    static EventLoopGroup getEventLoopGroup(boolean useEpoll, int threadCount, String threadNamePrefix, boolean boostIoRatio)
+    {
+        if (useEpoll)
+        {
+            logger.debug("using netty epoll event loop for pool prefix {}", threadNamePrefix);
+            EpollEventLoopGroup eventLoopGroup = new EpollEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix));
+            if (boostIoRatio)
+                eventLoopGroup.setIoRatio(100);
+            return eventLoopGroup;
+        }
+
+        logger.debug("using netty nio event loop for pool prefix {}", threadNamePrefix);
+        NioEventLoopGroup eventLoopGroup = new NioEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix));
+        if (boostIoRatio)
+            eventLoopGroup.setIoRatio(100);
+        return eventLoopGroup;
+    }
+
+    /**
+     * Create a {@link Channel} that listens on the {@code localAddr}. This method will block while trying to bind to the address,
+     * but it does not make a remote call.
+     */
+    public Channel createInboundChannel(InetSocketAddress localAddr, InboundInitializer initializer, int receiveBufferSize) throws ConfigurationException
+    {
+        String nic = FBUtilities.getNetworkInterface(localAddr.getAddress());
+        logger.info("Starting Messaging Service on {} {}, encryption: {}",
+                    localAddr, nic == null ? "" : String.format(" (%s)", nic), encryptionLogStatement(initializer.encryptionOptions));
+        Class<? extends ServerChannel> transport = useEpoll ? EpollServerSocketChannel.class : NioServerSocketChannel.class;
+        ServerBootstrap bootstrap = new ServerBootstrap().group(acceptGroup, inboundGroup)
+                                                         .channel(transport)
+                                                         .option(ChannelOption.SO_BACKLOG, 500)
+                                                         .childOption(ChannelOption.SO_KEEPALIVE, true)
+                                                         .childOption(ChannelOption.TCP_NODELAY, true)
+                                                         .childOption(ChannelOption.SO_REUSEADDR, true)
+                                                         .childOption(ChannelOption.SO_SNDBUF, INBOUND_CHANNEL_SEND_BUFFER_SIZE)
+                                                         .childHandler(initializer);
+
+        if (receiveBufferSize > 0)
+            bootstrap.childOption(ChannelOption.SO_RCVBUF, receiveBufferSize);
+
+        ChannelFuture channelFuture = bootstrap.bind(localAddr);
+
+        if (!channelFuture.awaitUninterruptibly().isSuccess())
+        {
+            if (channelFuture.channel().isOpen())
+                channelFuture.channel().close();
+
+            Throwable failedChannelCause = channelFuture.cause();
+
+            String causeString = "";
+            if (failedChannelCause != null && failedChannelCause.getMessage() != null)
+                causeString = failedChannelCause.getMessage();
+
+            if (causeString.contains("in use"))
+            {
+                throw new ConfigurationException(localAddr + " is in use by another process.  Change listen_address:storage_port " +
+                                                 "in cassandra.yaml to values that do not conflict with other services");
+            }
+            // looking at the jdk source, solaris/windows bind failue messages both use the phrase "cannot assign requested address".
+            // windows message uses "Cannot" (with a capital 'C'), and solaris (a/k/a *nux) doe not. hence we search for "annot" <sigh>
+            else if (causeString.contains("annot assign requested address"))
+            {
+                throw new ConfigurationException("Unable to bind to address " + localAddr
+                                                 + ". Set listen_address in cassandra.yaml to an interface you can bind to, e.g., your private IP address on EC2");
+            }
+            else
+            {
+                throw new ConfigurationException("failed to bind to: " + localAddr, failedChannelCause);
+            }
+        }
+
+        return channelFuture.channel();
+    }
+
+    public static class InboundInitializer extends ChannelInitializer<SocketChannel>
+    {
+        private final IInternodeAuthenticator authenticator;
+        private final ServerEncryptionOptions encryptionOptions;
+        private final ChannelGroup channelGroup;
+
+        public InboundInitializer(IInternodeAuthenticator authenticator, ServerEncryptionOptions encryptionOptions, ChannelGroup channelGroup)
+        {
+            this.authenticator = authenticator;
+            this.encryptionOptions = encryptionOptions;
+            this.channelGroup = channelGroup;
+        }
+
+        @Override
+        public void initChannel(SocketChannel channel) throws Exception
+        {
+            channelGroup.add(channel);
+            ChannelPipeline pipeline = channel.pipeline();
+
+            // order of handlers: ssl -> logger -> handshakeHandler
+            if (encryptionOptions != null)
+            {
+                SslContext sslContext = SSLFactory.getSslContext(encryptionOptions, true, true);
+                SslHandler sslHandler = sslContext.newHandler(channel.alloc());
+                logger.trace("creating inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
+                pipeline.addFirst(SSL_CHANNEL_HANDLER_NAME, sslHandler);            }
+
+            if (WIRETRACE)
+                pipeline.addLast("logger", new LoggingHandler(LogLevel.INFO));
+
+            channel.pipeline().addLast(HANDSHAKE_HANDLER_NAME, new InboundHandshakeHandler(authenticator));
+        }
+    }
+
+    private String encryptionLogStatement(ServerEncryptionOptions options)
+    {
+        if (options == null)
+            return "disabled";
+
+        String encryptionType = OpenSsl.isAvailable() ? "openssl" : "jdk";
+        return "enabled (" + encryptionType + ')';
+    }
+
+    /**
+     * Create the {@link Bootstrap} for connecting to a remote peer. This method does <b>not</b> attempt to connect to the peer,
+     * and thus does not block.
+     */
+    public Bootstrap createOutboundBootstrap(OutboundConnectionParams params)
+    {
+        logger.debug("creating outbound bootstrap to peer {}, compression: {}, encryption: {}, coalesce: {}", params.connectionId.connectionAddress(),
+                     params.compress, encryptionLogStatement(params.encryptionOptions),
+                     params.coalescingStrategy.isPresent() ? params.coalescingStrategy.get() : CoalescingStrategies.Strategy.DISABLED);
+        Class<? extends Channel>  transport = useEpoll ? EpollSocketChannel.class : NioSocketChannel.class;
+        Bootstrap bootstrap = new Bootstrap().group(outboundGroup)
+                              .channel(transport)
+                              .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 2000)
+                              .option(ChannelOption.SO_KEEPALIVE, true)
+                              .option(ChannelOption.SO_REUSEADDR, true)
+                              .option(ChannelOption.SO_SNDBUF, params.sendBufferSize)
+                              .option(ChannelOption.SO_RCVBUF, OUTBOUND_CHANNEL_RECEIVE_BUFFER_SIZE)
+                              .option(ChannelOption.TCP_NODELAY, params.tcpNoDelay)
+                              .option(ChannelOption.WRITE_BUFFER_WATER_MARK, params.waterMark)
+                              .handler(new OutboundInitializer(params));
+        bootstrap.localAddress(params.connectionId.local(), 0);
+        bootstrap.remoteAddress(params.connectionId.connectionAddress());
+        return bootstrap;
+    }
+
+    public static class OutboundInitializer extends ChannelInitializer<SocketChannel>
+    {
+        private final OutboundConnectionParams params;
+
+        OutboundInitializer(OutboundConnectionParams params)
+        {
+            this.params = params;
+        }
+
+        public void initChannel(SocketChannel channel) throws Exception
+        {
+            ChannelPipeline pipeline = channel.pipeline();
+
+            // order of handlers: ssl -> logger -> handshakeHandler
+            if (params.encryptionOptions != null)
+            {
+                SslContext sslContext = SSLFactory.getSslContext(params.encryptionOptions, true, false);
+
+                final SslHandler sslHandler;
+                if (params.encryptionOptions.require_endpoint_verification)
+                {
+                    InetSocketAddress peer = params.connectionId.remoteAddress();
+                    sslHandler = sslContext.newHandler(channel.alloc(), peer.getHostString(), peer.getPort());
+                    SSLEngine engine = sslHandler.engine();
+                    SSLParameters sslParameters = engine.getSSLParameters();
+                    sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
+                    engine.setSSLParameters(sslParameters);
+                }
+                else
+                {
+                    sslHandler = sslContext.newHandler(channel.alloc());
+                }
+
+                logger.trace("creating outbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
+                pipeline.addFirst(SSL_CHANNEL_HANDLER_NAME, sslHandler);
+            }
+
+            if (NettyFactory.WIRETRACE)
+                pipeline.addLast("logger", new LoggingHandler(LogLevel.INFO));
+
+            pipeline.addLast(HANDSHAKE_HANDLER_NAME, new OutboundHandshakeHandler(params));
+        }
+    }
+
+    public void close()
+    {
+        acceptGroup.shutdownGracefully();
+        outboundGroup.shutdownGracefully();
+        inboundGroup.shutdownGracefully();
+    }
+
+    static Lz4FrameEncoder createLz4Encoder(int protocolVersion)
+    {
+        return new Lz4FrameEncoder(LZ4Factory.fastestInstance(), false, COMPRESSION_BLOCK_SIZE, checksumForFrameEncoders(protocolVersion));
+    }
+
+    private static Checksum checksumForFrameEncoders(int protocolVersion)
+    {
+        if (protocolVersion >= MessagingService.current_version)
+            return ChecksumType.CRC32.newInstance();
+        return XXHashFactory.fastestInstance().newStreamingHash32(LZ4_HASH_SEED).asChecksum();
+    }
+
+    static Lz4FrameDecoder createLz4Decoder(int protocolVersion)
+    {
+        return new Lz4FrameDecoder(LZ4Factory.fastestInstance(), checksumForFrameEncoders(protocolVersion));
+    }
+
+    public static EventExecutor executorForChannelGroups()
+    {
+        return new DefaultEventExecutor();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java b/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
new file mode 100644
index 0000000..24dc5ff
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
@@ -0,0 +1,161 @@
+/*
+ * 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.async;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+
+/**
+ * Identifies an outbound messaging connection.
+ *
+ * This mainly hold the remote address and the type (small/large messages or gossip) of connection used, but with the
+ * additional detail that in some case (typically public EC2 address across regions) the address to which we connect
+ * to the remote is different from the address by which the node is known by the rest of the C*.
+ */
+public class OutboundConnectionIdentifier
+{
+    enum ConnectionType
+    {
+        GOSSIP, LARGE_MESSAGE, SMALL_MESSAGE
+    }
+
+    /**
+     * Memoization of the local node's broadcast address.
+     */
+    private final InetSocketAddress localAddr;
+
+    /**
+     * The address by which the remote is identified. This may be different from {@link #remoteConnectionAddr} for
+     * something like EC2 public IP address which need to be used for communication between EC2 regions.
+     */
+    private final InetSocketAddress remoteAddr;
+
+    /**
+     * The address to which we're connecting to the node (often the same as {@link #remoteAddr} but not always).
+     */
+    private final InetSocketAddress remoteConnectionAddr;
+
+    private final ConnectionType connectionType;
+
+    private OutboundConnectionIdentifier(InetSocketAddress localAddr,
+                                         InetSocketAddress remoteAddr,
+                                         InetSocketAddress remoteConnectionAddr,
+                                         ConnectionType connectionType)
+    {
+        this.localAddr = localAddr;
+        this.remoteAddr = remoteAddr;
+        this.remoteConnectionAddr = remoteConnectionAddr;
+        this.connectionType = connectionType;
+    }
+
+    private OutboundConnectionIdentifier(InetSocketAddress localAddr,
+                                         InetSocketAddress remoteAddr,
+                                         ConnectionType connectionType)
+    {
+        this(localAddr, remoteAddr, remoteAddr, connectionType);
+    }
+
+    /**
+     * Creates an identifier for a small message connection and using the remote "identifying" address as its connection
+     * address.
+     */
+    public static OutboundConnectionIdentifier small(InetSocketAddress localAddr, InetSocketAddress remoteAddr)
+    {
+        return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.SMALL_MESSAGE);
+    }
+
+    /**
+     * Creates an identifier for a large message connection and using the remote "identifying" address as its connection
+     * address.
+     */
+    public static OutboundConnectionIdentifier large(InetSocketAddress localAddr, InetSocketAddress remoteAddr)
+    {
+        return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.LARGE_MESSAGE);
+    }
+
+    /**
+     * Creates an identifier for a gossip connection and using the remote "identifying" address as its connection
+     * address.
+     */
+    public static OutboundConnectionIdentifier gossip(InetSocketAddress localAddr, InetSocketAddress remoteAddr)
+    {
+        return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.GOSSIP);
+    }
+
+    /**
+     * Returns a newly created connection identifier to the same remote that this identifier, but using the provided
+     * address as connection address.
+     *
+     * @param remoteConnectionAddr the address to use for connection to the remote in the new identifier.
+     * @return a newly created connection identifier that differs from this one only by using {@code remoteConnectionAddr}
+     * as connection address to the remote.
+     */
+    OutboundConnectionIdentifier withNewConnectionAddress(InetSocketAddress remoteConnectionAddr)
+    {
+        return new OutboundConnectionIdentifier(localAddr, remoteAddr, remoteConnectionAddr, connectionType);
+    }
+
+    /**
+     * The local node address.
+     */
+    InetAddress local()
+    {
+        return localAddr.getAddress();
+    }
+
+    /**
+     * The remote node identifying address (the one to use for anything else than connecting to the node).
+     */
+    InetSocketAddress remoteAddress()
+    {
+        return remoteAddr;
+    }
+
+    /**
+     * The remote node identifying address (the one to use for anything else than connecting to the node).
+     */
+    InetAddress remote()
+    {
+        return remoteAddr.getAddress();
+    }
+
+    /**
+     * The remote node connection address (the one to use to actually connect to the remote, and only that).
+     */
+    InetSocketAddress connectionAddress()
+    {
+        return remoteConnectionAddr;
+    }
+
+    /**
+     * The type of this connection.
+     */
+    ConnectionType type()
+    {
+        return connectionType;
+    }
+
+    @Override
+    public String toString()
+    {
+        return remoteAddr.equals(remoteConnectionAddr)
+               ? String.format("%s (%s)", remoteAddr, connectionType)
+               : String.format("%s on %s (%s)", remoteAddr, remoteConnectionAddr, connectionType);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java b/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java
new file mode 100644
index 0000000..282480e
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java
@@ -0,0 +1,202 @@
+/*
+ * 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.async;
+
+import java.util.Optional;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.base.Preconditions;
+
+import io.netty.channel.WriteBufferWaterMark;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult;
+import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
+
+/**
+ * A collection of data points to be passed around for outbound connections.
+ */
+public class OutboundConnectionParams
+{
+    public static final int DEFAULT_SEND_BUFFER_SIZE = 1 << 16;
+
+    final OutboundConnectionIdentifier connectionId;
+    final Consumer<HandshakeResult> callback;
+    final ServerEncryptionOptions encryptionOptions;
+    final NettyFactory.Mode mode;
+    final boolean compress;
+    final Optional<CoalescingStrategy> coalescingStrategy;
+    final int sendBufferSize;
+    final boolean tcpNoDelay;
+    final Supplier<QueuedMessage> backlogSupplier;
+    final Consumer<MessageResult> messageResultConsumer;
+    final WriteBufferWaterMark waterMark;
+    final int protocolVersion;
+
+    private OutboundConnectionParams(OutboundConnectionIdentifier connectionId,
+                                     Consumer<HandshakeResult> callback,
+                                     ServerEncryptionOptions encryptionOptions,
+                                     NettyFactory.Mode mode,
+                                     boolean compress,
+                                     Optional<CoalescingStrategy> coalescingStrategy,
+                                     int sendBufferSize,
+                                     boolean tcpNoDelay,
+                                     Supplier<QueuedMessage> backlogSupplier,
+                                     Consumer<MessageResult> messageResultConsumer,
+                                     WriteBufferWaterMark waterMark,
+                                     int protocolVersion)
+    {
+        this.connectionId = connectionId;
+        this.callback = callback;
+        this.encryptionOptions = encryptionOptions;
+        this.mode = mode;
+        this.compress = compress;
+        this.coalescingStrategy = coalescingStrategy;
+        this.sendBufferSize = sendBufferSize;
+        this.tcpNoDelay = tcpNoDelay;
+        this.backlogSupplier = backlogSupplier;
+        this.messageResultConsumer = messageResultConsumer;
+        this.waterMark = waterMark;
+        this.protocolVersion = protocolVersion;
+    }
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static Builder builder(OutboundConnectionParams params)
+    {
+        return new Builder(params);
+    }
+    
+    public static class Builder
+    {
+        private OutboundConnectionIdentifier connectionId;
+        private Consumer<HandshakeResult> callback;
+        private ServerEncryptionOptions encryptionOptions;
+        private NettyFactory.Mode mode;
+        private boolean compress;
+        private Optional<CoalescingStrategy> coalescingStrategy = Optional.empty();
+        private int sendBufferSize = DEFAULT_SEND_BUFFER_SIZE;
+        private boolean tcpNoDelay;
+        private Supplier<QueuedMessage> backlogSupplier;
+        private Consumer<MessageResult> messageResultConsumer;
+        private WriteBufferWaterMark waterMark = WriteBufferWaterMark.DEFAULT;
+        int protocolVersion;
+
+        private Builder()
+        {   }
+
+        private Builder(OutboundConnectionParams params)
+        {
+            this.connectionId = params.connectionId;
+            this.callback = params.callback;
+            this.encryptionOptions = params.encryptionOptions;
+            this.mode = params.mode;
+            this.compress = params.compress;
+            this.coalescingStrategy = params.coalescingStrategy;
+            this.sendBufferSize = params.sendBufferSize;
+            this.tcpNoDelay = params.tcpNoDelay;
+            this.backlogSupplier = params.backlogSupplier;
+            this.messageResultConsumer = params.messageResultConsumer;
+        }
+
+        public Builder connectionId(OutboundConnectionIdentifier connectionId)
+        {
+            this.connectionId = connectionId;
+            return this;
+        }
+
+        public Builder callback(Consumer<HandshakeResult> callback)
+        {
+            this.callback = callback;
+            return this;
+        }
+
+        public Builder encryptionOptions(ServerEncryptionOptions encryptionOptions)
+        {
+            this.encryptionOptions = encryptionOptions;
+            return this;
+        }
+
+        public Builder mode(NettyFactory.Mode mode)
+        {
+            this.mode = mode;
+            return this;
+        }
+
+        public Builder compress(boolean compress)
+        {
+            this.compress = compress;
+            return this;
+        }
+
+        public Builder coalescingStrategy(Optional<CoalescingStrategy> coalescingStrategy)
+        {
+            this.coalescingStrategy = coalescingStrategy;
+            return this;
+        }
+
+        public Builder sendBufferSize(int sendBufferSize)
+        {
+            this.sendBufferSize = sendBufferSize;
+            return this;
+        }
+
+        public Builder tcpNoDelay(boolean tcpNoDelay)
+        {
+            this.tcpNoDelay = tcpNoDelay;
+            return this;
+        }
+
+        public Builder backlogSupplier(Supplier<QueuedMessage> backlogSupplier)
+        {
+            this.backlogSupplier = backlogSupplier;
+            return this;
+        }
+
+        public Builder messageResultConsumer(Consumer<MessageResult> messageResultConsumer)
+        {
+            this.messageResultConsumer = messageResultConsumer;
+            return this;
+        }
+
+        public Builder waterMark(WriteBufferWaterMark waterMark)
+        {
+            this.waterMark = waterMark;
+            return this;
+        }
+
+        public Builder protocolVersion(int protocolVersion)
+        {
+            this.protocolVersion = protocolVersion;
+            return this;
+        }
+
+        public OutboundConnectionParams build()
+        {
+            Preconditions.checkArgument(protocolVersion > 0, "illegal protocol version: " + protocolVersion);
+            Preconditions.checkArgument(sendBufferSize > 0 && sendBufferSize < 1 << 20, "illegal send buffer size: " + sendBufferSize);
+
+            return new OutboundConnectionParams(connectionId, callback, encryptionOptions, mode, compress, coalescingStrategy, sendBufferSize,
+                                                tcpNoDelay, backlogSupplier, messageResultConsumer, waterMark, protocolVersion);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java b/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
new file mode 100644
index 0000000..703549a
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
@@ -0,0 +1,255 @@
+/*
+ * 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.async;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandler;
+import io.netty.channel.ChannelPipeline;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.concurrent.Future;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage;
+import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage;
+import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage;
+
+import static org.apache.cassandra.config.Config.PROPERTY_PREFIX;
+
+/**
+ * A {@link ChannelHandler} to execute the send-side of the internode communication handshake protocol.
+ * As soon as the handler is added to the channel via {@link #channelActive(ChannelHandlerContext)}
+ * (which is only invoked if the underlying TCP connection was properly established), the {@link FirstHandshakeMessage}
+ * of the internode messaging protocol is automatically sent out. See {@link HandshakeProtocol} for full details
+ * about the internode messaging hndshake protocol.
+ * <p>
+ * Upon completion of the handshake (on success or fail), the {@link #callback} is invoked to let the listener
+ * know the result of the handshake. See {@link HandshakeResult} for details about the different result states.
+ * <p>
+ * This class extends {@link ByteToMessageDecoder}, which is a {@link ChannelInboundHandler}, because this handler
+ * waits for the peer's handshake response (the {@link SecondHandshakeMessage} of the internode messaging handshake protocol).
+ */
+public class OutboundHandshakeHandler extends ByteToMessageDecoder
+{
+    private static final Logger logger = LoggerFactory.getLogger(OutboundHandshakeHandler.class);
+
+    /**
+     * The number of milliseconds to wait before closing a channel if there has been no progress (when there is
+     * data to be sent).See {@link IdleStateHandler} and {@link MessageOutHandler#userEventTriggered(ChannelHandlerContext, Object)}.
+     */
+    private static final long DEFAULT_WRITE_IDLE_MS = TimeUnit.SECONDS.toMillis(10);
+    private static final String WRITE_IDLE_PROPERTY = PROPERTY_PREFIX + "outbound_write_idle_ms";
+    private static final long WRITE_IDLE_MS = Long.getLong(WRITE_IDLE_PROPERTY, DEFAULT_WRITE_IDLE_MS);
+
+    private final OutboundConnectionIdentifier connectionId;
+
+    /**
+     * The expected messaging service version to use.
+     */
+    private final int messagingVersion;
+
+    /**
+     * A function to invoke upon completion of the attempt, success or failure, to connect to the peer.
+     */
+    private final Consumer<HandshakeResult> callback;
+    private final NettyFactory.Mode mode;
+    private final OutboundConnectionParams params;
+
+    OutboundHandshakeHandler(OutboundConnectionParams params)
+    {
+        this.params = params;
+        this.connectionId = params.connectionId;
+        this.messagingVersion = params.protocolVersion;
+        this.callback = params.callback;
+        this.mode = params.mode;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Invoked when the channel is made active, and sends out the {@link FirstHandshakeMessage}
+     */
+    @Override
+    public void channelActive(final ChannelHandlerContext ctx) throws Exception
+    {
+        FirstHandshakeMessage msg = new FirstHandshakeMessage(messagingVersion, mode, params.compress);
+        logger.trace("starting handshake with peer {}, msg = {}", connectionId.connectionAddress(), msg);
+        ctx.writeAndFlush(msg.encode(ctx.alloc())).addListener(future -> firstHandshakeMessageListener(future, ctx));
+        ctx.fireChannelActive();
+    }
+
+    /**
+     * A simple listener to make sure we could send the {@link FirstHandshakeMessage} to the socket,
+     * and fail the handshake attempt if we could not (for example, maybe we could create the TCP socket, but then
+     * the connection gets closed for some reason).
+     */
+    void firstHandshakeMessageListener(Future<? super Void> future, ChannelHandlerContext ctx)
+    {
+        if (future.isSuccess())
+            return;
+
+        ChannelFuture channelFuture = (ChannelFuture)future;
+        exceptionCaught(ctx, channelFuture.cause());
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Invoked when we get the response back from the peer, which should contain the second message of the internode messaging handshake.
+     * <p>
+     * If the peer's protocol version does not equal what we were expecting, immediately close the channel (and socket);
+     * do *not* send out the third message of the internode messaging handshake.
+     * We will reconnect on the appropriate protocol version.
+     */
+    @Override
+    protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception
+    {
+        SecondHandshakeMessage msg = SecondHandshakeMessage.maybeDecode(in);
+        if (msg == null)
+            return;
+
+        logger.trace("received second handshake message from peer {}, msg = {}", connectionId.connectionAddress(), msg);
+        final int peerMessagingVersion = msg.messagingVersion;
+
+        // we expected a higher protocol version, but it was actually lower
+        if (messagingVersion > peerMessagingVersion)
+        {
+            logger.trace("peer's max version is {}; will reconnect with that version", peerMessagingVersion);
+            try
+            {
+                if (DatabaseDescriptor.getSeeds().contains(connectionId.remote()))
+                    logger.warn("Seed gossip version is {}; will not connect with that version", peerMessagingVersion);
+            }
+            catch (Throwable e)
+            {
+                // If invalid yaml has been added to the config since startup, getSeeds() will throw an AssertionError
+                // Additionally, third party seed providers may throw exceptions if network is flakey.
+                // Regardless of what's thrown, we must catch it, disconnect, and try again
+                logger.warn("failed to reread yaml (on trying to connect to a seed): {}", e.getLocalizedMessage());
+            }
+            ctx.close();
+            callback.accept(HandshakeResult.disconnect(peerMessagingVersion));
+            return;
+        }
+        // we anticipate a version that is lower than what peer is actually running
+        else if (messagingVersion < peerMessagingVersion && messagingVersion < MessagingService.current_version)
+        {
+            logger.trace("peer has a higher max version than expected {} (previous value {})", peerMessagingVersion, messagingVersion);
+            ctx.close();
+            callback.accept(HandshakeResult.disconnect(peerMessagingVersion));
+            return;
+        }
+
+        try
+        {
+            ctx.writeAndFlush(new ThirdHandshakeMessage(MessagingService.current_version, connectionId.local()).encode(ctx.alloc()));
+            ChannelWriter channelWriter = setupPipeline(ctx.channel(), peerMessagingVersion);
+            callback.accept(HandshakeResult.success(channelWriter, peerMessagingVersion));
+        }
+        catch (Exception e)
+        {
+            logger.info("failed to finalize internode messaging handshake", e);
+            ctx.close();
+            callback.accept(HandshakeResult.failed());
+        }
+    }
+
+    @VisibleForTesting
+    ChannelWriter setupPipeline(Channel channel, int messagingVersion)
+    {
+        ChannelPipeline pipeline = channel.pipeline();
+        pipeline.addLast("idleWriteHandler", new IdleStateHandler(true, 0, WRITE_IDLE_MS, 0, TimeUnit.MILLISECONDS));
+        if (params.compress)
+            pipeline.addLast(NettyFactory.OUTBOUND_COMPRESSOR_HANDLER_NAME, NettyFactory.createLz4Encoder(messagingVersion));
+
+        ChannelWriter channelWriter = ChannelWriter.create(channel, params.messageResultConsumer, params.coalescingStrategy);
+        pipeline.addLast("messageOutHandler", new MessageOutHandler(connectionId, messagingVersion, channelWriter, params.backlogSupplier));
+        pipeline.remove(this);
+        return channelWriter;
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+    {
+        logger.error("Failed to properly handshake with peer {}. Closing the channel.", connectionId, cause);
+        ctx.close();
+        callback.accept(HandshakeResult.failed());
+    }
+
+    /**
+     * The result of the handshake. Handshake has 3 possible outcomes:
+     *  1) it can be successful, in which case the channel and version to used is returned in this result.
+     *  2) we may decide to disconnect to reconnect with another protocol version (namely, the version is passed in this result).
+     *  3) we can have a negotiation failure for an unknown reason. (#sadtrombone)
+     */
+    public static class HandshakeResult
+    {
+        static final int UNKNOWN_PROTOCOL_VERSION = -1;
+
+        /**
+         * Describes the result of receiving the response back from the peer (Message 2 of the handshake)
+         * and implies an action that should be taken.
+         */
+        enum Outcome
+        {
+            SUCCESS, DISCONNECT, NEGOTIATION_FAILURE
+        }
+
+        /** The channel for the connection, only set for successful handshake. */
+        final ChannelWriter channelWriter;
+        /** The version negotiated with the peer. Set unless this is a {@link Outcome#NEGOTIATION_FAILURE}. */
+        final int negotiatedMessagingVersion;
+        /** The handshake {@link Outcome}. */
+        final Outcome outcome;
+
+        private HandshakeResult(ChannelWriter channelWriter, int negotiatedMessagingVersion, Outcome outcome)
+        {
+            this.channelWriter = channelWriter;
+            this.negotiatedMessagingVersion = negotiatedMessagingVersion;
+            this.outcome = outcome;
+        }
+
+        static HandshakeResult success(ChannelWriter channel, int negotiatedMessagingVersion)
+        {
+            return new HandshakeResult(channel, negotiatedMessagingVersion, Outcome.SUCCESS);
+        }
+
+        static HandshakeResult disconnect(int negotiatedMessagingVersion)
+        {
+            return new HandshakeResult(null, negotiatedMessagingVersion, Outcome.DISCONNECT);
+        }
+
+        static HandshakeResult failed()
+        {
+            return new HandshakeResult(null, UNKNOWN_PROTOCOL_VERSION, Outcome.NEGOTIATION_FAILURE);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java b/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
new file mode 100644
index 0000000..6bda9cd
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
@@ -0,0 +1,716 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.Future;
+import org.apache.cassandra.auth.IInternodeAuthenticator;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.NettyFactory.Mode;
+import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult;
+import org.apache.cassandra.utils.CoalescingStrategies;
+import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.NoSpamLogger;
+
+/**
+ * Represents one connection to a peer, and handles the state transistions on the connection and the netty {@link Channel}
+ * The underlying socket is not opened until explicitly requested (by sending a message).
+ *
+ * The basic setup for the channel is like this: a message is requested to be sent via {@link #sendMessage(MessageOut, int)}.
+ * If the channel is not established, then we need to create it (obviously). To prevent multiple threads from creating
+ * independent connections, they attempt to update the {@link #state}; one thread will win the race and create the connection.
+ * Upon sucessfully setting up the connection/channel, the {@link #state} will be updated again (to {@link State#READY},
+ * which indicates to other threads that the channel is ready for business and can be written to.
+ *
+ */
+public class OutboundMessagingConnection
+{
+    static final Logger logger = LoggerFactory.getLogger(OutboundMessagingConnection.class);
+    private static final NoSpamLogger errorLogger = NoSpamLogger.getLogger(logger, 10, TimeUnit.SECONDS);
+
+    private static final String INTRADC_TCP_NODELAY_PROPERTY = Config.PROPERTY_PREFIX + "otc_intradc_tcp_nodelay";
+
+    /**
+     * Enabled/disable TCP_NODELAY for intradc connections. Defaults to enabled.
+     */
+    private static final boolean INTRADC_TCP_NODELAY = Boolean.parseBoolean(System.getProperty(INTRADC_TCP_NODELAY_PROPERTY, "true"));
+
+    /**
+     * Number of milliseconds between connection createRetry attempts.
+     */
+    private static final int OPEN_RETRY_DELAY_MS = 100;
+
+    /**
+     * A minimum number of milliseconds to wait for a connection (TCP socket connect + handshake)
+     */
+    private static final int MINIMUM_CONNECT_TIMEOUT_MS = 2000;
+    private final IInternodeAuthenticator authenticator;
+
+    /**
+     * Describes this instance's ability to send messages into it's Netty {@link Channel}.
+     */
+    enum State
+    {
+        /** waiting to create the connection */
+        NOT_READY,
+        /** we've started to create the connection/channel */
+        CREATING_CHANNEL,
+        /** channel is established and we can send messages */
+        READY,
+        /** a dead state which should not be transitioned away from */
+        CLOSED
+    }
+
+    /**
+     * Backlog to hold messages passed by upstream threads while the Netty {@link Channel} is being set up or recreated.
+     */
+    private final Queue<QueuedMessage> backlog;
+
+    /**
+     * Reference to a {@link ScheduledExecutorService} rther than directly depending on something like {@link ScheduledExecutors}.
+     */
+    private final ScheduledExecutorService scheduledExecutor;
+
+    final AtomicLong droppedMessageCount;
+    final AtomicLong completedMessageCount;
+
+    private volatile OutboundConnectionIdentifier connectionId;
+
+    private final ServerEncryptionOptions encryptionOptions;
+
+    /**
+     * A future for retrying connections. Bear in mind that this future does not execute in the
+     * netty event event loop, so there's some races to be careful of.
+     */
+    private volatile ScheduledFuture<?> connectionRetryFuture;
+
+    /**
+     * A future for notifying when the timeout for creating the connection and negotiating the handshake has elapsed.
+     * It will be cancelled when the channel is established correctly. Bear in mind that this future does not execute in the
+     * netty event event loop, so there's some races to be careful of.
+     */
+    private volatile ScheduledFuture<?> connectionTimeoutFuture;
+
+    private final AtomicReference<State> state;
+
+    private final Optional<CoalescingStrategy> coalescingStrategy;
+
+    /**
+     * A running count of the number of times we've tried to create a connection.
+     */
+    private volatile int connectAttemptCount;
+
+    /**
+     * The netty channel, once a socket connection is established; it won't be in it's normal working state until the handshake is complete.
+     */
+    private volatile ChannelWriter channelWriter;
+
+    /**
+     * the target protocol version to communicate to the peer with, discovered/negotiated via handshaking
+     */
+    private int targetVersion;
+
+    OutboundMessagingConnection(OutboundConnectionIdentifier connectionId,
+                                ServerEncryptionOptions encryptionOptions,
+                                Optional<CoalescingStrategy> coalescingStrategy,
+                                IInternodeAuthenticator authenticator)
+    {
+        this(connectionId, encryptionOptions, coalescingStrategy, authenticator, ScheduledExecutors.scheduledFastTasks);
+    }
+
+    @VisibleForTesting
+    OutboundMessagingConnection(OutboundConnectionIdentifier connectionId,
+                                ServerEncryptionOptions encryptionOptions,
+                                Optional<CoalescingStrategy> coalescingStrategy,
+                                IInternodeAuthenticator authenticator,
+                                ScheduledExecutorService sceduledExecutor)
+    {
+        this.connectionId = connectionId;
+        this.encryptionOptions = encryptionOptions;
+        this.authenticator = authenticator;
+        backlog = new ConcurrentLinkedQueue<>();
+        droppedMessageCount = new AtomicLong(0);
+        completedMessageCount = new AtomicLong(0);
+        state = new AtomicReference<>(State.NOT_READY);
+        this.scheduledExecutor = sceduledExecutor;
+        this.coalescingStrategy = coalescingStrategy;
+
+        // We want to use the most precise protocol version we know because while there is version detection on connect(),
+        // the target version might be accessed by the pool (in getConnection()) before we actually connect (as we
+        // only connect when the first message is submitted). Note however that the only case where we'll connect
+        // without knowing the true version of a node is if that node is a seed (otherwise, we can't know a node
+        // unless it has been gossiped to us or it has connected to us, and in both cases that will set the version).
+        // In that case we won't rely on that targetVersion before we're actually connected and so the version
+        // detection in connect() will do its job.
+        targetVersion = MessagingService.instance().getVersion(connectionId.remote());
+    }
+
+    /**
+     * If the connection is set up and ready to use (the normal case), simply send the message to it and return.
+     * Otherwise, one lucky thread is selected to create the Channel, while other threads just add the {@code msg} to
+     * the backlog queue.
+     *
+     * @return true if the message was accepted by the {@link #channelWriter}; else false if it was not accepted
+     * and added to the backlog or the channel is {@link State#CLOSED}. See documentation in {@link ChannelWriter} and
+     * {@link MessageOutHandler} how the backlogged messages get consumed.
+     */
+    boolean sendMessage(MessageOut msg, int id)
+    {
+        return sendMessage(new QueuedMessage(msg, id));
+    }
+
+    boolean sendMessage(QueuedMessage queuedMessage)
+    {
+        State state = this.state.get();
+        if (state == State.READY)
+        {
+            if (channelWriter.write(queuedMessage, false))
+                return true;
+
+            backlog.add(queuedMessage);
+            return false;
+        }
+        else if (state == State.CLOSED)
+        {
+            errorLogger.warn("trying to write message to a closed connection");
+            return false;
+        }
+        else
+        {
+            backlog.add(queuedMessage);
+            connect();
+            return true;
+        }
+    }
+
+    /**
+     * Initiate all the actions required to establish a working, valid connection. This includes
+     * opening the socket, negotiating the internode messaging handshake, and setting up the working
+     * Netty {@link Channel}. However, this method will not block for all those actions: it will only
+     * kick off the connection attempt as everything is asynchronous.
+     * <p>
+     * Threads compete to update the {@link #state} field to {@link State#CREATING_CHANNEL} to ensure only one
+     * connection is attempted at a time.
+     *
+     * @return true if kicking off the connection attempt was started by this thread; else, false.
+     */
+    public boolean connect()
+    {
+        // try to be the winning thread to create the channel
+        if (!state.compareAndSet(State.NOT_READY, State.CREATING_CHANNEL))
+            return false;
+
+        // clean up any lingering connection attempts
+        if (connectionTimeoutFuture != null)
+        {
+            connectionTimeoutFuture.cancel(false);
+            connectionTimeoutFuture = null;
+        }
+
+        return tryConnect();
+    }
+
+    private boolean tryConnect()
+    {
+        if (state.get() != State.CREATING_CHANNEL)
+                return false;
+
+        logger.debug("connection attempt {} to {}", connectAttemptCount, connectionId);
+
+
+        InetSocketAddress remote = connectionId.remoteAddress();
+        if (!authenticator.authenticate(remote.getAddress(), remote.getPort()))
+        {
+            logger.warn("Internode auth failed connecting to {}", connectionId);
+            //Remove the connection pool and other thread so messages aren't queued
+            MessagingService.instance().destroyConnectionPool(remote.getAddress());
+
+            // don't update the state field as destroyConnectionPool() *should* call OMC.close()
+            // on all the connections in the OMP for the remoteAddress
+            return false;
+        }
+
+        boolean compress = shouldCompressConnection(connectionId.local(), connectionId.remote());
+        Bootstrap bootstrap = buildBootstrap(compress);
+
+        ChannelFuture connectFuture = bootstrap.connect();
+        connectFuture.addListener(this::connectCallback);
+
+        long timeout = Math.max(MINIMUM_CONNECT_TIMEOUT_MS, DatabaseDescriptor.getRpcTimeout());
+        if (connectionTimeoutFuture == null || connectionTimeoutFuture.isDone())
+            connectionTimeoutFuture = scheduledExecutor.schedule(() -> connectionTimeout(connectFuture), timeout, TimeUnit.MILLISECONDS);
+        return true;
+    }
+
+    @VisibleForTesting
+    static boolean shouldCompressConnection(InetAddress localHost, InetAddress remoteHost)
+    {
+        return (DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.all)
+               || ((DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.dc) && !isLocalDC(localHost, remoteHost));
+    }
+
+    private Bootstrap buildBootstrap(boolean compress)
+    {
+        boolean tcpNoDelay = isLocalDC(connectionId.local(), connectionId.remote()) ? INTRADC_TCP_NODELAY : DatabaseDescriptor.getInterDCTcpNoDelay();
+        int sendBufferSize = DatabaseDescriptor.getInternodeSendBufferSize() > 0
+                             ? DatabaseDescriptor.getInternodeSendBufferSize()
+                             : OutboundConnectionParams.DEFAULT_SEND_BUFFER_SIZE;
+        OutboundConnectionParams params = OutboundConnectionParams.builder()
+                                                                  .connectionId(connectionId)
+                                                                  .callback(this::finishHandshake)
+                                                                  .encryptionOptions(encryptionOptions)
+                                                                  .mode(Mode.MESSAGING)
+                                                                  .compress(compress)
+                                                                  .coalescingStrategy(coalescingStrategy)
+                                                                  .sendBufferSize(sendBufferSize)
+                                                                  .tcpNoDelay(tcpNoDelay)
+                                                                  .backlogSupplier(() -> nextBackloggedMessage())
+                                                                  .messageResultConsumer(this::handleMessageResult)
+                                                                  .protocolVersion(targetVersion)
+                                                                  .build();
+
+        return NettyFactory.instance.createOutboundBootstrap(params);
+    }
+
+    private QueuedMessage nextBackloggedMessage()
+    {
+        QueuedMessage msg = backlog.poll();
+        if (msg == null)
+            return null;
+
+        if (!msg.isTimedOut())
+            return msg;
+
+        if (msg.shouldRetry())
+            return msg.createRetry();
+
+        droppedMessageCount.incrementAndGet();
+        return null;
+    }
+
+    static boolean isLocalDC(InetAddress localHost, InetAddress remoteHost)
+    {
+        String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(remoteHost);
+        String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(localHost);
+        return remoteDC != null && remoteDC.equals(localDC);
+    }
+
+    /**
+     * Handles the callback of the TCP connection attempt (not including the handshake negotiation!), and really all
+     * we're handling here is the TCP connection failures. On failure, we close the channel (which should disconnect
+     * the socket, if connected). If there was an {@link IOException} while trying to connect, the connection will be
+     * retried after a short delay.
+     * <p>
+     * This method does not alter the {@link #state} as it's only evaluating the TCP connect, not TCP connect and handshake.
+     * Thus, {@link #finishHandshake(HandshakeResult)} will handle any necessary state updates.
+     * <p>
+     * Note: this method is called from the event loop, so be careful wrt thread visibility
+     *
+     * @return true iff the TCP connection was established and the {@link #state} is not {@link State#CLOSED}; else false.
+     */
+    @VisibleForTesting
+    boolean connectCallback(Future<? super Void> future)
+    {
+        ChannelFuture channelFuture = (ChannelFuture)future;
+
+        // make sure this instance is not (terminally) closed
+        if (state.get() == State.CLOSED)
+        {
+            channelFuture.channel().close();
+            return false;
+        }
+
+        // this is the success state
+        final Throwable cause = future.cause();
+        if (cause == null)
+        {
+            connectAttemptCount = 0;
+            return true;
+        }
+
+        setStateIfNotClosed(state, State.NOT_READY);
+        if (cause instanceof IOException)
+        {
+            logger.trace("unable to connect on attempt {} to {}", connectAttemptCount, connectionId, cause);
+            connectAttemptCount++;
+            connectionRetryFuture = scheduledExecutor.schedule(this::connect, OPEN_RETRY_DELAY_MS * connectAttemptCount, TimeUnit.MILLISECONDS);
+        }
+        else
+        {
+            JVMStabilityInspector.inspectThrowable(cause);
+            logger.error("non-IO error attempting to connect to {}", connectionId, cause);
+        }
+        return false;
+    }
+
+    /**
+     * A callback for handling timeouts when creating a connection/negotiating the handshake.
+     * <p>
+     * Note: this method is *not* invoked from the netty event loop,
+     * so there's an inherent race with {@link #finishHandshake(HandshakeResult)},
+     * as well as any possible connect() reattempts (a seemingly remote race condition, however).
+     * Therefore, this function tries to lose any races, as much as possible.
+     *
+     * @return true if there was a timeout on the connect/handshake; else false.
+     */
+    boolean connectionTimeout(ChannelFuture channelFuture)
+    {
+        if (connectionRetryFuture != null)
+        {
+            connectionRetryFuture.cancel(false);
+            connectionRetryFuture = null;
+        }
+        connectAttemptCount = 0;
+        State initialState = state.get();
+        if (initialState == State.CLOSED)
+            return true;
+
+        if (initialState != State.READY)
+        {
+            logger.debug("timed out while trying to connect to {}", connectionId);
+
+            channelFuture.channel().close();
+            // a last-ditch attempt to let finishHandshake() win the race
+            if (state.compareAndSet(initialState, State.NOT_READY))
+            {
+                backlog.clear();
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Process the results of the handshake negotiation.
+     * <p>
+     * Note: this method will be invoked from the netty event loop,
+     * so there's an inherent race with {@link #connectionTimeout(ChannelFuture)}.
+     */
+    void finishHandshake(HandshakeResult result)
+    {
+        // clean up the connector instances before changing the state
+        if (connectionTimeoutFuture != null)
+        {
+            connectionTimeoutFuture.cancel(false);
+            connectionTimeoutFuture = null;
+        }
+        if (connectionRetryFuture != null)
+        {
+            connectionRetryFuture.cancel(false);
+            connectionRetryFuture = null;
+        }
+        connectAttemptCount = 0;
+
+        if (result.negotiatedMessagingVersion != HandshakeResult.UNKNOWN_PROTOCOL_VERSION)
+        {
+            targetVersion = result.negotiatedMessagingVersion;
+            MessagingService.instance().setVersion(connectionId.remote(), targetVersion);
+        }
+
+        switch (result.outcome)
+        {
+            case SUCCESS:
+                assert result.channelWriter != null;
+                logger.debug("successfully connected to {}, conmpress = {}, coalescing = {}", connectionId,
+                             shouldCompressConnection(connectionId.local(), connectionId.remote()),
+                             coalescingStrategy.isPresent() ? coalescingStrategy.get() : CoalescingStrategies.Strategy.DISABLED);
+                if (state.get() == State.CLOSED)
+                {
+                    result.channelWriter.close();
+                    backlog.clear();
+                    break;
+                }
+                channelWriter = result.channelWriter;
+                // drain the backlog to the channel
+                channelWriter.writeBacklog(backlog, true);
+                // change the state so newly incoming messages can be sent to the channel (without adding to the backlog)
+                setStateIfNotClosed(state, State.READY);
+                // ship out any stragglers that got added to the backlog
+                channelWriter.writeBacklog(backlog, true);
+                break;
+            case DISCONNECT:
+                reconnect();
+                break;
+            case NEGOTIATION_FAILURE:
+                setStateIfNotClosed(state, State.NOT_READY);
+                backlog.clear();
+                break;
+            default:
+                throw new IllegalArgumentException("unhandled result type: " + result.outcome);
+        }
+    }
+
+    @VisibleForTesting
+    static boolean setStateIfNotClosed(AtomicReference<State> state, State newState)
+    {
+        State s = state.get();
+        if (s == State.CLOSED)
+            return false;
+        state.set(newState);
+        return true;
+    }
+
+    int getTargetVersion()
+    {
+        return targetVersion;
+    }
+
+    /**
+     * Handles the result of each message sent.
+     *
+     * Note: this function is expected to be invoked on the netty event loop. Also, do not retain any state from
+     * the input {@code messageResult}.
+     */
+    void handleMessageResult(MessageResult messageResult)
+    {
+        completedMessageCount.incrementAndGet();
+
+        // checking the cause() is an optimized way to tell if the operation was successful (as the cause will be null)
+        // Note that ExpiredException is just a marker for timeout-ed message we're dropping, but as we already
+        // incremented the dropped message count in MessageOutHandler, we have nothing to do.
+        Throwable cause = messageResult.future.cause();
+        if (cause == null)
+            return;
+
+        if (cause instanceof ExpiredException)
+        {
+            droppedMessageCount.incrementAndGet();
+            return;
+        }
+
+        JVMStabilityInspector.inspectThrowable(cause);
+
+        if (cause instanceof IOException || cause.getCause() instanceof IOException)
+        {
+            ChannelWriter writer = messageResult.writer;
+            if (writer.shouldPurgeBacklog())
+                purgeBacklog();
+
+            // This writer needs to be closed and we need to trigger a reconnection. We really only want to do that
+            // once for this channel however (and again, no race because we're on the netty event loop).
+            if (!writer.isClosed() && messageResult.allowReconnect)
+            {
+                reconnect();
+                writer.close();
+            }
+
+            QueuedMessage msg = messageResult.msg;
+            if (msg != null && msg.shouldRetry())
+            {
+                sendMessage(msg.createRetry());
+            }
+        }
+        else if (messageResult.future.isCancelled())
+        {
+            // Someone cancelled the future, which we assume meant it doesn't want the message to be sent if it hasn't
+            // yet. Just ignore.
+        }
+        else
+        {
+            // Non IO exceptions are likely a programming error so let's not silence them
+            logger.error("Unexpected error writing on " + connectionId, cause);
+        }
+    }
+
+    /**
+     * Change the IP address on which we connect to the peer. We will attempt to connect to the new address if there
+     * was a previous connection, and new incoming messages as well as existing {@link #backlog} messages will be sent there.
+     * Any outstanding messages in the existing channel will still be sent to the previous address (we won't/can't move them from
+     * one channel to another).
+     */
+    void reconnectWithNewIp(InetSocketAddress newAddr)
+    {
+        State currentState = state.get();
+
+        // if we're closed, ignore the request
+        if (currentState == State.CLOSED)
+            return;
+
+        // capture a reference to the current channel, in case it gets swapped out before we can call close() on it
+        ChannelWriter currentChannel = channelWriter;
+        connectionId = connectionId.withNewConnectionAddress(newAddr);
+
+        if (currentState != State.NOT_READY)
+            reconnect();
+
+        // lastly, push through anything remaining in the existing channel.
+        if (currentChannel != null)
+            currentChannel.softClose();
+    }
+
+    /**
+     * Sets the state properly so {@link #connect()} can attempt to reconnect.
+     */
+    void reconnect()
+    {
+        if (setStateIfNotClosed(state, State.NOT_READY))
+            connect();
+    }
+
+    void purgeBacklog()
+    {
+        backlog.clear();
+    }
+
+    public void close(boolean softClose)
+    {
+        state.set(State.CLOSED);
+
+        if (connectionTimeoutFuture != null)
+        {
+            connectionTimeoutFuture.cancel(false);
+            connectionTimeoutFuture = null;
+        }
+
+        // drain the backlog
+        if (channelWriter != null)
+        {
+            if (softClose)
+            {
+                channelWriter.writeBacklog(backlog, false);
+                channelWriter.softClose();
+            }
+            else
+            {
+                backlog.clear();
+                channelWriter.close();
+            }
+
+            channelWriter = null;
+        }
+    }
+
+    @Override
+    public String toString()
+    {
+        return connectionId.toString();
+    }
+
+    public Integer getPendingMessages()
+    {
+        int pending = backlog.size();
+        ChannelWriter chan = channelWriter;
+        if (chan != null)
+            pending += (int)chan.pendingMessageCount();
+        return pending;
+    }
+
+    public Long getCompletedMessages()
+    {
+        return completedMessageCount.get();
+    }
+
+    public Long getDroppedMessages()
+    {
+        return droppedMessageCount.get();
+    }
+
+    /*
+        methods specific to testing follow
+     */
+
+    @VisibleForTesting
+    int backlogSize()
+    {
+        return backlog.size();
+    }
+
+    @VisibleForTesting
+    void addToBacklog(QueuedMessage msg)
+    {
+        backlog.add(msg);
+    }
+
+    @VisibleForTesting
+    void setChannelWriter(ChannelWriter channelWriter)
+    {
+        this.channelWriter = channelWriter;
+    }
+
+    @VisibleForTesting
+    ChannelWriter getChannelWriter()
+    {
+        return channelWriter;
+    }
+
+    @VisibleForTesting
+    void setState(State state)
+    {
+        this.state.set(state);
+    }
+
+    @VisibleForTesting
+    State getState()
+    {
+        return state.get();
+    }
+
+    @VisibleForTesting
+    void setTargetVersion(int targetVersion)
+    {
+        this.targetVersion = targetVersion;
+    }
+
+    @VisibleForTesting
+    OutboundConnectionIdentifier getConnectionId()
+    {
+        return connectionId;
+    }
+
+    @VisibleForTesting
+    void setConnectionTimeoutFuture(ScheduledFuture<?> connectionTimeoutFuture)
+    {
+        this.connectionTimeoutFuture = connectionTimeoutFuture;
+    }
+
+    @VisibleForTesting
+    ScheduledFuture<?> getConnectionTimeoutFuture()
+    {
+        return connectionTimeoutFuture;
+    }
+
+    public boolean isConnected()
+    {
+        return state.get() == State.READY;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java b/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java
new file mode 100644
index 0000000..0086da8
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java
@@ -0,0 +1,173 @@
+/*
+ * 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.async;
+
+import java.net.InetSocketAddress;
+import java.util.Optional;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.auth.IInternodeAuthenticator;
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.apache.cassandra.metrics.ConnectionMetrics;
+import org.apache.cassandra.net.BackPressureState;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType;
+import org.apache.cassandra.utils.CoalescingStrategies;
+import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
+
+/**
+ * Groups a set of outbound connections to a given peer, and routes outgoing messages to the appropriate connection
+ * (based upon message's type or size). Contains a {@link OutboundMessagingConnection} for each of the
+ * {@link ConnectionType} type.
+ */
+public class OutboundMessagingPool
+{
+    @VisibleForTesting
+    static final long LARGE_MESSAGE_THRESHOLD = Long.getLong(Config.PROPERTY_PREFIX + "otcp_large_message_threshold", 1024 * 64);
+
+    private final ConnectionMetrics metrics;
+    private final BackPressureState backPressureState;
+
+    public OutboundMessagingConnection gossipChannel;
+    public OutboundMessagingConnection largeMessageChannel;
+    public OutboundMessagingConnection smallMessageChannel;
+
+    /**
+     * An override address on which to communicate with the peer. Typically used for something like EC2 public IP addresses
+     * which need to be used for communication between EC2 regions.
+     */
+    private InetSocketAddress preferredRemoteAddr;
+
+    public OutboundMessagingPool(InetSocketAddress remoteAddr, InetSocketAddress localAddr, ServerEncryptionOptions encryptionOptions,
+                                 BackPressureState backPressureState, IInternodeAuthenticator authenticator)
+    {
+        preferredRemoteAddr = remoteAddr;
+        this.backPressureState = backPressureState;
+        metrics = new ConnectionMetrics(localAddr.getAddress(), this);
+
+
+        smallMessageChannel = new OutboundMessagingConnection(OutboundConnectionIdentifier.small(localAddr, preferredRemoteAddr),
+                                                              encryptionOptions, coalescingStrategy(remoteAddr), authenticator);
+        largeMessageChannel = new OutboundMessagingConnection(OutboundConnectionIdentifier.large(localAddr, preferredRemoteAddr),
+                                                              encryptionOptions, coalescingStrategy(remoteAddr), authenticator);
+
+        // don't attempt coalesce the gossip messages, just ship them out asap (let's not anger the FD on any peer node by any artificial delays)
+        gossipChannel = new OutboundMessagingConnection(OutboundConnectionIdentifier.gossip(localAddr, preferredRemoteAddr),
+                                                        encryptionOptions, Optional.empty(), authenticator);
+    }
+
+    private static Optional<CoalescingStrategy> coalescingStrategy(InetSocketAddress remoteAddr)
+    {
+        String strategyName = DatabaseDescriptor.getOtcCoalescingStrategy();
+        String displayName = remoteAddr.getAddress().getHostAddress();
+        return CoalescingStrategies.newCoalescingStrategy(strategyName,
+                                                          DatabaseDescriptor.getOtcCoalescingWindow(),
+                                                          OutboundMessagingConnection.logger,
+                                                          displayName);
+
+    }
+
+    public BackPressureState getBackPressureState()
+    {
+        return backPressureState;
+    }
+
+    public void sendMessage(MessageOut msg, int id)
+    {
+        getConnection(msg).sendMessage(msg, id);
+    }
+
+    @VisibleForTesting
+    public OutboundMessagingConnection getConnection(MessageOut msg)
+    {
+        // optimize for the common path (the small message channel)
+        if (Stage.GOSSIP != msg.getStage())
+        {
+            return msg.serializedSize(smallMessageChannel.getTargetVersion()) < LARGE_MESSAGE_THRESHOLD
+            ? smallMessageChannel
+            : largeMessageChannel;
+        }
+        return gossipChannel;
+    }
+
+    /**
+     * Reconnect to the peer using the given {@code addr}. Outstanding messages in each channel will be sent on the
+     * current channel. Typically this function is used for something like EC2 public IP addresses which need to be used
+     * for communication between EC2 regions.
+     *
+     * @param addr IP Address to use (and prefer) going forward for connecting to the peer
+     */
+    public void reconnectWithNewIp(InetSocketAddress addr)
+    {
+        preferredRemoteAddr = addr;
+        gossipChannel.reconnectWithNewIp(addr);
+        largeMessageChannel.reconnectWithNewIp(addr);
+        smallMessageChannel.reconnectWithNewIp(addr);
+    }
+
+    /**
+     * Close each netty channel and it's socket.
+     *
+     * @param softClose {@code true} if existing messages in the queue should be sent before closing.
+     */
+    public void close(boolean softClose)
+    {
+        gossipChannel.close(softClose);
+        largeMessageChannel.close(softClose);
+        smallMessageChannel.close(softClose);
+    }
+
+    /**
+     * For testing purposes only.
+     */
+    @VisibleForTesting
+    OutboundMessagingConnection getConnection(ConnectionType connectionType)
+    {
+        switch (connectionType)
+        {
+            case GOSSIP:
+                return gossipChannel;
+            case LARGE_MESSAGE:
+                return largeMessageChannel;
+            case SMALL_MESSAGE:
+                return smallMessageChannel;
+            default:
+                throw new IllegalArgumentException("unsupported connection type: " + connectionType);
+        }
+    }
+
+    public void incrementTimeout()
+    {
+        metrics.timeouts.mark();
+    }
+
+    public long getTimeouts()
+    {
+        return metrics.timeouts.getCount();
+    }
+
+    public InetSocketAddress getPreferredRemoteAddr()
+    {
+        return preferredRemoteAddr;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/async/QueuedMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/QueuedMessage.java b/src/java/org/apache/cassandra/net/async/QueuedMessage.java
new file mode 100644
index 0000000..28e4ba4
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/async/QueuedMessage.java
@@ -0,0 +1,75 @@
+/*
+ * 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.async;
+
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.CoalescingStrategies;
+
+/**
+ *  A wrapper for outbound messages. All messages will be retried once.
+ */
+public class QueuedMessage implements CoalescingStrategies.Coalescable
+{
+    public final MessageOut<?> message;
+    public final int id;
+    public final long timestampNanos;
+    public final boolean droppable;
+    private final boolean retryable;
+
+    public QueuedMessage(MessageOut<?> message, int id)
+    {
+        this(message, id, System.nanoTime(), MessagingService.DROPPABLE_VERBS.contains(message.verb), true);
+    }
+
+    @VisibleForTesting
+    public QueuedMessage(MessageOut<?> message, int id, long timestampNanos, boolean droppable, boolean retryable)
+    {
+        this.message = message;
+        this.id = id;
+        this.timestampNanos = timestampNanos;
+        this.droppable = droppable;
+        this.retryable = retryable;
+    }
+
+    /** don't drop a non-droppable message just because it's timestamp is expired */
+    public boolean isTimedOut()
+    {
+        return droppable && timestampNanos < System.nanoTime() - TimeUnit.MILLISECONDS.toNanos(message.getTimeout());
+    }
+
+    public boolean shouldRetry()
+    {
+        return retryable;
+    }
+
+    public QueuedMessage createRetry()
+    {
+        return new QueuedMessage(message, id, System.nanoTime(), droppable, false);
+    }
+
+    public long timestampNanos()
+    {
+        return timestampNanos;
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org

[11/11] cassandra git commit: switch internode messaging to netty

Posted by ja...@apache.org.
switch internode messaging to netty

patch by jasobrown, reviewed by pcmanus for CASSANDRA-8457


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

Branch: refs/heads/trunk
Commit: 356dc3c253224751cbf80b32cfce4e3c1640de11
Parents: 3d4a7e7
Author: Jason Brown <ja...@gmail.com>
Authored: Mon Feb 8 07:04:00 2016 -0800
Committer: Jason Brown <ja...@gmail.com>
Committed: Tue Aug 22 13:54:44 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 build.xml                                       |   2 +-
 conf/cassandra-env.sh                           |   1 +
 lib/licenses/netty-4.1.14.txt                   | 202 ++++++
 lib/licenses/netty-all-4.0.44.Final.txt         | 202 ------
 lib/netty-all-4.0.44.Final.jar                  | Bin 2342652 -> 0 bytes
 lib/netty-all-4.1.14.Final.jar                  | Bin 0 -> 3690637 bytes
 .../org/apache/cassandra/config/Config.java     |   7 +-
 .../cassandra/config/DatabaseDescriptor.java    |  15 +
 .../cassandra/config/EncryptionOptions.java     |   4 +-
 src/java/org/apache/cassandra/db/TypeSizes.java |   6 +
 .../cassandra/locator/PropertyFileSnitch.java   |   2 +-
 .../locator/ReconnectableSnitchHelper.java      |  10 +-
 .../cassandra/metrics/ConnectionMetrics.java    |  27 +-
 .../cassandra/net/IncomingTcpConnection.java    | 197 -----
 .../org/apache/cassandra/net/MessageIn.java     |  35 +-
 .../org/apache/cassandra/net/MessageOut.java    | 128 +++-
 .../apache/cassandra/net/MessagingService.java  | 577 +++++++--------
 .../cassandra/net/OutboundTcpConnection.java    | 693 ------------------
 .../net/OutboundTcpConnectionPool.java          | 229 ------
 .../net/async/ByteBufDataInputPlus.java         |  31 +
 .../net/async/ByteBufDataOutputPlus.java        | 140 ++++
 .../cassandra/net/async/ChannelWriter.java      | 418 +++++++++++
 .../cassandra/net/async/ExpiredException.java   |  28 +
 .../cassandra/net/async/HandshakeProtocol.java  | 304 ++++++++
 .../net/async/InboundHandshakeHandler.java      | 293 ++++++++
 .../cassandra/net/async/MessageInHandler.java   | 314 ++++++++
 .../cassandra/net/async/MessageOutHandler.java  | 324 +++++++++
 .../cassandra/net/async/MessageResult.java      |  51 ++
 .../cassandra/net/async/NettyFactory.java       | 375 ++++++++++
 .../net/async/OutboundConnectionIdentifier.java | 161 +++++
 .../net/async/OutboundConnectionParams.java     | 202 ++++++
 .../net/async/OutboundHandshakeHandler.java     | 255 +++++++
 .../net/async/OutboundMessagingConnection.java  | 716 +++++++++++++++++++
 .../net/async/OutboundMessagingPool.java        | 173 +++++
 .../cassandra/net/async/QueuedMessage.java      |  75 ++
 .../apache/cassandra/security/SSLFactory.java   | 222 +++---
 .../streaming/DefaultConnectionFactory.java     |  31 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../apache/cassandra/tracing/TracingImpl.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   4 +-
 .../org/apache/cassandra/transport/Server.java  |  25 +-
 .../cassandra/transport/SimpleClient.java       |  18 +-
 .../cassandra/utils/CoalescingStrategies.java   | 406 ++++-------
 .../org/apache/cassandra/utils/FBUtilities.java |   7 +
 .../apache/cassandra/utils/NativeLibrary.java   |   2 +-
 test/conf/cassandra_ssl_test.keystore           | Bin 0 -> 2281 bytes
 test/conf/cassandra_ssl_test.truststore         | Bin 0 -> 992 bytes
 .../apache/cassandra/db/ReadCommandTest.java    |  33 +
 .../apache/cassandra/locator/EC2SnitchTest.java |  20 -
 .../cassandra/net/MessagingServiceTest.java     | 120 +++-
 .../net/OutboundTcpConnectionTest.java          | 175 -----
 .../net/async/ByteBufDataOutputPlusTest.java    | 178 +++++
 .../cassandra/net/async/ChannelWriterTest.java  | 312 ++++++++
 .../net/async/HandshakeHandlersTest.java        | 204 ++++++
 .../net/async/HandshakeProtocolTest.java        |  95 +++
 .../net/async/InboundHandshakeHandlerTest.java  | 289 ++++++++
 .../net/async/MessageInHandlerTest.java         | 242 +++++++
 .../net/async/MessageOutHandlerTest.java        | 289 ++++++++
 .../cassandra/net/async/NettyFactoryTest.java   | 300 ++++++++
 .../NonSendingOutboundMessagingConnection.java  |  42 ++
 .../net/async/OutboundConnectionParamsTest.java |  36 +
 .../net/async/OutboundHandshakeHandlerTest.java | 209 ++++++
 .../async/OutboundMessagingConnectionTest.java  | 519 ++++++++++++++
 .../net/async/OutboundMessagingPoolTest.java    | 149 ++++
 .../cassandra/net/async/TestAuthenticator.java  |  42 ++
 .../RepairMessageSerializationsTest.java        |   2 +
 .../cassandra/security/SSLFactoryTest.java      | 136 +++-
 .../streaming/StreamingTransferTest.java        |  29 +-
 .../utils/CoalescingStrategiesTest.java         | 453 ++----------
 70 files changed, 8024 insertions(+), 2769 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 75a4be9..f2e643e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Use netty for internode messaging (CASSANDRA-8457)
  * Add bytes repaired/unrepaired to nodetool tablestats (CASSANDRA-13774)
  * Don't delete incremental repair sessions if they still have sstables (CASSANDRA-13758)
  * Fix pending repair manager index out of bounds check (CASSANDRA-13769)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index e033bb6..ee22921 100644
--- a/build.xml
+++ b/build.xml
@@ -420,7 +420,7 @@
           <dependency groupId="com.addthis.metrics" artifactId="reporter-config3" version="3.0.3" />
           <dependency groupId="org.mindrot" artifactId="jbcrypt" version="0.3m" />
           <dependency groupId="io.airlift" artifactId="airline" version="0.7" />
-          <dependency groupId="io.netty" artifactId="netty-all" version="4.0.44.Final" />
+          <dependency groupId="io.netty" artifactId="netty-all" version="4.1.13.Final" />
           <dependency groupId="com.google.code.findbugs" artifactId="jsr305" version="2.0.2" />
           <dependency groupId="com.clearspring.analytics" artifactId="stream" version="2.5.2" />
           <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" version="3.0.1" classifier="shaded">

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index 5a02f79..347fbf3 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -293,3 +293,4 @@ JVM_OPTS="$JVM_OPTS -Djava.library.path=$CASSANDRA_HOME/lib/sigar-bin"
 JVM_OPTS="$JVM_OPTS $MX4J_ADDRESS"
 JVM_OPTS="$JVM_OPTS $MX4J_PORT"
 JVM_OPTS="$JVM_OPTS $JVM_EXTRA_OPTS"
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/lib/licenses/netty-4.1.14.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/netty-4.1.14.txt b/lib/licenses/netty-4.1.14.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/lib/licenses/netty-4.1.14.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/lib/licenses/netty-all-4.0.44.Final.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/netty-all-4.0.44.Final.txt b/lib/licenses/netty-all-4.0.44.Final.txt
deleted file mode 100644
index d645695..0000000
--- a/lib/licenses/netty-all-4.0.44.Final.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/lib/netty-all-4.0.44.Final.jar
----------------------------------------------------------------------
diff --git a/lib/netty-all-4.0.44.Final.jar b/lib/netty-all-4.0.44.Final.jar
deleted file mode 100644
index 9c5bda5..0000000
Binary files a/lib/netty-all-4.0.44.Final.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/lib/netty-all-4.1.14.Final.jar
----------------------------------------------------------------------
diff --git a/lib/netty-all-4.1.14.Final.jar b/lib/netty-all-4.1.14.Final.jar
new file mode 100644
index 0000000..e5c8137
Binary files /dev/null and b/lib/netty-all-4.1.14.Final.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 5a45282..77d5bf4 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -276,6 +276,7 @@ public class Config
     public int tracetype_repair_ttl = (int) TimeUnit.DAYS.toSeconds(7);
 
     /**
+<<<<<<< HEAD
      * Maintain statistics on whether writes achieve the ideal consistency level
      * before expiring and becoming hints
      */
@@ -283,8 +284,12 @@ public class Config
 
     /*
      * Strategy to use for coalescing messages in OutboundTcpConnection.
+=======
+     * Strategy to use for coalescing messages in {@link OutboundMessagingPool}.
+>>>>>>> 2acc8dbc74... switch internode messaging to netty
      * Can be fixed, movingaverage, timehorizon, disabled. Setting is case and leading/trailing
-     * whitespace insensitive. You can also specify a subclass of CoalescingStrategies.CoalescingStrategy by name.
+     * whitespace insensitive. You can also specify a subclass of
+     * {@link org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy} by name.
      */
     public String otc_coalescing_strategy = "DISABLED";
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index fb50826..53bac93 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1567,6 +1567,11 @@ public class DatabaseDescriptor
         return conf.listen_on_broadcast_address;
     }
 
+    public static void setListenOnBroadcastAddress(boolean listen_on_broadcast_address)
+    {
+        conf.listen_on_broadcast_address = listen_on_broadcast_address;
+    }
+
     public static IInternodeAuthenticator getInternodeAuthenticator()
     {
         return internodeAuthenticator;
@@ -2095,6 +2100,11 @@ public class DatabaseDescriptor
         return conf.internode_compression;
     }
 
+    public static void setInternodeCompression(Config.InternodeCompression compression)
+    {
+        conf.internode_compression = compression;
+    }
+
     public static boolean getInterDCTcpNoDelay()
     {
         return conf.inter_dc_tcp_nodelay;
@@ -2156,6 +2166,11 @@ public class DatabaseDescriptor
         return conf.otc_coalescing_strategy;
     }
 
+    public static void setOtcCoalescingStrategy(String strategy)
+    {
+        conf.otc_coalescing_strategy = strategy;
+    }
+
     public static int getOtcCoalescingWindow()
     {
         return conf.otc_coalescing_window_us;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/config/EncryptionOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java
index d662871..6010746 100644
--- a/src/java/org/apache/cassandra/config/EncryptionOptions.java
+++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java
@@ -17,15 +17,13 @@
  */
 package org.apache.cassandra.config;
 
-import javax.net.ssl.SSLSocketFactory;
-
 public abstract class EncryptionOptions
 {
     public String keystore = "conf/.keystore";
     public String keystore_password = "cassandra";
     public String truststore = "conf/.truststore";
     public String truststore_password = "cassandra";
-    public String[] cipher_suites = ((SSLSocketFactory)SSLSocketFactory.getDefault()).getDefaultCipherSuites();
+    public String[] cipher_suites = {};
     public String protocol = "TLS";
     public String algorithm = "SunX509";
     public String store_type = "JKS";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/db/TypeSizes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/TypeSizes.java b/src/java/org/apache/cassandra/db/TypeSizes.java
index 6543025..b47e300 100644
--- a/src/java/org/apache/cassandra/db/TypeSizes.java
+++ b/src/java/org/apache/cassandra/db/TypeSizes.java
@@ -28,6 +28,7 @@ public final class TypeSizes
     private TypeSizes(){}
 
     private static final int BOOL_SIZE = 1;
+    private static final int BYTE_SIZE = 1;
     private static final int SHORT_SIZE = 2;
     private static final int INT_SIZE = 4;
     private static final int LONG_SIZE = 8;
@@ -78,6 +79,11 @@ public final class TypeSizes
         return BOOL_SIZE;
     }
 
+    public static int sizeof(byte value)
+    {
+        return BYTE_SIZE;
+    }
+
     public static int sizeof(short value)
     {
         return SHORT_SIZE;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
index 8cc6549..2908976 100644
--- a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
@@ -192,7 +192,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
             throw new ConfigurationException(String.format("Snitch definitions at %s do not define a location for " +
                                                            "this node's broadcast address %s, nor does it provides a default",
                                                            SNITCH_PROPERTIES_FILENAME, broadcastAddress));
-        // OutboundTcpConnectionPool.getEndpoint() converts our broadcast address to local,
+        // internode messaging code converts our broadcast address to local,
         // make sure we can be found at that as well.
         InetAddress localAddress = FBUtilities.getLocalAddress();
         if (!localAddress.equals(broadcastAddress) && !reloadedMap.containsKey(localAddress))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
index 08f0a14..2235c76 100644
--- a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
+++ b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
@@ -23,9 +23,9 @@ import java.net.UnknownHostException;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.gms.*;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.OutboundTcpConnectionPool;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -64,18 +64,16 @@ public class ReconnectableSnitchHelper implements IEndpointStateChangeSubscriber
     @VisibleForTesting
     static void reconnect(InetAddress publicAddress, InetAddress localAddress, IEndpointSnitch snitch, String localDc)
     {
-        OutboundTcpConnectionPool cp = MessagingService.instance().getConnectionPool(publicAddress);
-        //InternodeAuthenticator said don't connect
-        if (cp == null)
+        if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(publicAddress, MessagingService.portFor(publicAddress)))
         {
             logger.debug("InternodeAuthenticator said don't reconnect to {} on {}", publicAddress, localAddress);
             return;
         }
 
         if (snitch.getDatacenter(publicAddress).equals(localDc)
-                && !cp.endPoint().equals(localAddress))
+                && !MessagingService.instance().getCurrentEndpoint(publicAddress).equals(localAddress))
         {
-            cp.reset(localAddress);
+            MessagingService.instance().reconnectWithNewIp(publicAddress, localAddress);
             logger.debug("Initiated reconnect to an Internal IP {} for the {}", localAddress, publicAddress);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
index f01c06d..7815784 100644
--- a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
@@ -21,14 +21,12 @@ import java.net.InetAddress;
 
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Meter;
+import org.apache.cassandra.net.async.OutboundMessagingPool;
 
 import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
-
-import org.apache.cassandra.net.OutboundTcpConnectionPool;
-
 /**
- * Metrics for {@link OutboundTcpConnectionPool}.
+ * Metrics for internode connections.
  */
 public class ConnectionMetrics
 {
@@ -66,9 +64,8 @@ public class ConnectionMetrics
      * Create metrics for given connection pool.
      *
      * @param ip IP address to use for metrics label
-     * @param connectionPool Connection pool
      */
-    public ConnectionMetrics(InetAddress ip, final OutboundTcpConnectionPool connectionPool)
+    public ConnectionMetrics(InetAddress ip, final OutboundMessagingPool messagingPool)
     {
         // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName
         address = ip.getHostAddress().replace(':', '.');
@@ -79,63 +76,63 @@ public class ConnectionMetrics
         {
             public Integer getValue()
             {
-                return connectionPool.largeMessages.getPendingMessages();
+                return messagingPool.largeMessageChannel.getPendingMessages();
             }
         });
         largeMessageCompletedTasks = Metrics.register(factory.createMetricName("LargeMessageCompletedTasks"), new Gauge<Long>()
         {
             public Long getValue()
             {
-                return connectionPool.largeMessages.getCompletedMesssages();
+                return messagingPool.largeMessageChannel.getCompletedMessages();
             }
         });
         largeMessageDroppedTasks = Metrics.register(factory.createMetricName("LargeMessageDroppedTasks"), new Gauge<Long>()
         {
             public Long getValue()
             {
-                return connectionPool.largeMessages.getDroppedMessages();
+                return messagingPool.largeMessageChannel.getDroppedMessages();
             }
         });
         smallMessagePendingTasks = Metrics.register(factory.createMetricName("SmallMessagePendingTasks"), new Gauge<Integer>()
         {
             public Integer getValue()
             {
-                return connectionPool.smallMessages.getPendingMessages();
+                return messagingPool.smallMessageChannel.getPendingMessages();
             }
         });
         smallMessageCompletedTasks = Metrics.register(factory.createMetricName("SmallMessageCompletedTasks"), new Gauge<Long>()
         {
             public Long getValue()
             {
-                return connectionPool.smallMessages.getCompletedMesssages();
+                return messagingPool.smallMessageChannel.getCompletedMessages();
             }
         });
         smallMessageDroppedTasks = Metrics.register(factory.createMetricName("SmallMessageDroppedTasks"), new Gauge<Long>()
         {
             public Long getValue()
             {
-                return connectionPool.smallMessages.getDroppedMessages();
+                return messagingPool.smallMessageChannel.getDroppedMessages();
             }
         });
         gossipMessagePendingTasks = Metrics.register(factory.createMetricName("GossipMessagePendingTasks"), new Gauge<Integer>()
         {
             public Integer getValue()
             {
-                return connectionPool.gossipMessages.getPendingMessages();
+                return messagingPool.gossipChannel.getPendingMessages();
             }
         });
         gossipMessageCompletedTasks = Metrics.register(factory.createMetricName("GossipMessageCompletedTasks"), new Gauge<Long>()
         {
             public Long getValue()
             {
-                return connectionPool.gossipMessages.getCompletedMesssages();
+                return messagingPool.gossipChannel.getCompletedMessages();
             }
         });
         gossipMessageDroppedTasks = Metrics.register(factory.createMetricName("GossipMessageDroppedTasks"), new Gauge<Long>()
         {
             public Long getValue()
             {
-                return connectionPool.gossipMessages.getDroppedMessages();
+                return messagingPool.gossipChannel.getDroppedMessages();
             }
         });
         timeouts = Metrics.meter(factory.createMetricName("Timeouts"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/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
deleted file mode 100644
index 67e54c8..0000000
--- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
+++ /dev/null
@@ -1,197 +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.net;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.net.SocketException;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.util.zip.Checksum;
-import java.util.Set;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.netty.util.concurrent.FastThreadLocalThread;
-import net.jpountz.lz4.LZ4BlockInputStream;
-import net.jpountz.lz4.LZ4FastDecompressor;
-import net.jpountz.lz4.LZ4Factory;
-import net.jpountz.xxhash.XXHashFactory;
-
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.exceptions.UnknownTableException;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.monitoring.ApproximateTime;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
-import org.apache.cassandra.io.util.NIODataInputStream;
-
-public class IncomingTcpConnection extends FastThreadLocalThread implements Closeable
-{
-    private static final Logger logger = LoggerFactory.getLogger(IncomingTcpConnection.class);
-
-    private static final int BUFFER_SIZE = Integer.getInteger(Config.PROPERTY_PREFIX + ".itc_buffer_size", 1024 * 4);
-
-    private final int version;
-    private final boolean compressed;
-    private final Socket socket;
-    private final Set<Closeable> group;
-    public InetAddress from;
-
-    public IncomingTcpConnection(int version, boolean compressed, Socket socket, Set<Closeable> group)
-    {
-        super("MessagingService-Incoming-" + socket.getInetAddress());
-        this.version = version;
-        this.compressed = compressed;
-        this.socket = socket;
-        this.group = group;
-        if (DatabaseDescriptor.getInternodeRecvBufferSize() > 0)
-        {
-            try
-            {
-                this.socket.setReceiveBufferSize(DatabaseDescriptor.getInternodeRecvBufferSize());
-            }
-            catch (SocketException se)
-            {
-                logger.warn("Failed to set receive buffer size on internode socket.", se);
-            }
-        }
-    }
-
-    /**
-     * A new connection will either stream or message for its entire lifetime: because streaming
-     * bypasses the InputStream implementations to use sendFile, we cannot begin buffering until
-     * we've determined the type of the connection.
-     */
-    @Override
-    public void run()
-    {
-        try
-        {
-            if (version < MessagingService.VERSION_30)
-                throw new UnsupportedOperationException(String.format("Unable to read obsolete message version %s; "
-                                                                      + "The earliest version supported is 3.0.0",
-                                                                      version));
-
-            receiveMessages();
-        }
-        catch (EOFException e)
-        {
-            logger.trace("eof reading from socket; closing", e);
-            // connection will be reset so no need to throw an exception.
-        }
-        catch (UnknownTableException e)
-        {
-            logger.warn("UnknownTableException reading from socket; closing", e);
-        }
-        catch (IOException e)
-        {
-            logger.trace("IOException reading from socket; closing", e);
-        }
-        finally
-        {
-            close();
-        }
-    }
-
-    @Override
-    public void close()
-    {
-        try
-        {
-            if (logger.isTraceEnabled())
-                logger.trace("Closing socket {} - isclosed: {}", socket, socket.isClosed());
-            if (!socket.isClosed())
-            {
-                socket.close();
-            }
-        }
-        catch (IOException e)
-        {
-            logger.trace("Error closing socket", e);
-        }
-        finally
-        {
-            group.remove(this);
-        }
-    }
-
-    @SuppressWarnings("resource") // Not closing constructed DataInputPlus's as the stream needs to remain open.
-    private void receiveMessages() throws IOException
-    {
-        // handshake (true) endpoint versions
-        DataOutputStream out = new DataOutputStream(socket.getOutputStream());
-        // if this version is < the MS version the other node is trying
-        // to connect with, the other node will disconnect
-        out.writeInt(MessagingService.current_version);
-        out.flush();
-        DataInputPlus in = new DataInputStreamPlus(socket.getInputStream());
-        int maxVersion = in.readInt();
-        // outbound side will reconnect if necessary to upgrade version
-        assert version <= MessagingService.current_version;
-        from = CompactEndpointSerializationHelper.deserialize(in);
-        // record the (true) version of the endpoint
-        MessagingService.instance().setVersion(from, maxVersion);
-        logger.trace("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from));
-
-        if (compressed)
-        {
-            logger.trace("Upgrading incoming connection to be compressed");
-            LZ4FastDecompressor decompressor = LZ4Factory.fastestInstance().fastDecompressor();
-            Checksum checksum = XXHashFactory.fastestInstance().newStreamingHash32(OutboundTcpConnection.LZ4_HASH_SEED).asChecksum();
-            in = new DataInputStreamPlus(new LZ4BlockInputStream(socket.getInputStream(),
-                                                             decompressor,
-                                                             checksum));
-        }
-        else
-        {
-            ReadableByteChannel channel = socket.getChannel();
-            in = new NIODataInputStream(channel != null ? channel : Channels.newChannel(socket.getInputStream()), BUFFER_SIZE);
-        }
-
-        while (true)
-        {
-            MessagingService.validateMagic(in.readInt());
-            receiveMessage(in, version);
-        }
-    }
-
-    private InetAddress receiveMessage(DataInputPlus input, int version) throws IOException
-    {
-        int id = input.readInt();
-
-        long currentTime = ApproximateTime.currentTimeMillis();
-        MessageIn message = MessageIn.read(input, version, id, MessageIn.readConstructionTime(from, input, currentTime));
-        if (message == null)
-        {
-            // callback expired; nothing to do
-            return null;
-        }
-        if (version <= MessagingService.current_version)
-        {
-            MessagingService.instance().receive(message, id);
-        }
-        else
-        {
-            logger.trace("Received connection from newer protocol version {}. Ignoring message", version);
-        }
-        return message.from;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/MessageIn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessageIn.java b/src/java/org/apache/cassandra/net/MessageIn.java
index 8774d38..d520fa9 100644
--- a/src/java/org/apache/cassandra/net/MessageIn.java
+++ b/src/java/org/apache/cassandra/net/MessageIn.java
@@ -31,20 +31,27 @@ import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.net.MessagingService.Verb;
 
+/**
+ * The receiving node's view of a {@link MessageOut}. See documentation on {@link MessageOut} for details on the
+ * serialization format.
+ *
+ * @param <T> The type of the payload
+ */
 public class MessageIn<T>
 {
     public final InetAddress from;
     public final T payload;
     public final Map<String, byte[]> parameters;
-    public final MessagingService.Verb verb;
+    public final Verb verb;
     public final int version;
     public final long constructionTime;
 
     private MessageIn(InetAddress from,
                       T payload,
                       Map<String, byte[]> parameters,
-                      MessagingService.Verb verb,
+                      Verb verb,
                       int version,
                       long constructionTime)
     {
@@ -59,7 +66,7 @@ public class MessageIn<T>
     public static <T> MessageIn<T> create(InetAddress from,
                                           T payload,
                                           Map<String, byte[]> parameters,
-                                          MessagingService.Verb verb,
+                                          Verb verb,
                                           int version,
                                           long constructionTime)
     {
@@ -85,11 +92,17 @@ public class MessageIn<T>
         InetAddress from = CompactEndpointSerializationHelper.deserialize(in);
 
         MessagingService.Verb verb = MessagingService.Verb.fromId(in.readInt());
+        Map<String, byte[]> parameters = readParameters(in);
+        int payloadSize = in.readInt();
+        return read(in, version, id, constructionTime, from, payloadSize, verb, parameters);
+    }
+
+    public static Map<String, byte[]> readParameters(DataInputPlus in) throws IOException
+    {
         int parameterCount = in.readInt();
-        Map<String, byte[]> parameters;
         if (parameterCount == 0)
         {
-            parameters = Collections.emptyMap();
+            return Collections.emptyMap();
         }
         else
         {
@@ -101,10 +114,13 @@ public class MessageIn<T>
                 in.readFully(value);
                 builder.put(key, value);
             }
-            parameters = builder.build();
+            return builder.build();
         }
+    }
 
-        int payloadSize = in.readInt();
+    public static <T2> MessageIn<T2> read(DataInputPlus in, int version, int id, long constructionTime,
+                                          InetAddress from, int payloadSize, Verb verb, Map<String, byte[]> parameters) throws IOException
+    {
         IVersionedSerializer<T2> serializer = (IVersionedSerializer<T2>) MessagingService.verbSerializers.get(verb);
         if (serializer instanceof MessagingService.CallbackDeterminedSerializer)
         {
@@ -124,12 +140,11 @@ public class MessageIn<T>
         return MessageIn.create(from, payload, parameters, verb, version, constructionTime);
     }
 
-    public static long readConstructionTime(InetAddress from, DataInputPlus input, long currentTime) throws IOException
+    public static long deriveConstructionTime(InetAddress from, int messageTimestamp, long currentTime)
     {
         // Reconstruct the message construction time sent by the remote host (we sent only the lower 4 bytes, assuming the
         // higher 4 bytes wouldn't change between the sender and receiver)
-        int partial = input.readInt(); // make sure to readInt, even if cross_node_to is not enabled
-        long sentConstructionTime = (currentTime & 0xFFFFFFFF00000000L) | (((partial & 0xFFFFFFFFL) << 2) >> 2);
+        long sentConstructionTime = (currentTime & 0xFFFFFFFF00000000L) | (((messageTimestamp & 0xFFFFFFFFL) << 2) >> 2);
 
         // Because nodes may not have their clock perfectly in sync, it's actually possible the sentConstructionTime is
         // later than the currentTime (the received time). If that's the case, as we definitively know there is a lack

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/net/MessageOut.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessageOut.java b/src/java/org/apache/cassandra/net/MessageOut.java
index a38aed5..379aff5 100644
--- a/src/java/org/apache/cassandra/net/MessageOut.java
+++ b/src/java/org/apache/cassandra/net/MessageOut.java
@@ -29,21 +29,78 @@ import com.google.common.collect.ImmutableMap;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+
 import static org.apache.cassandra.tracing.Tracing.isTracing;
 
+/**
+ * Each message contains a header with several fixed fields, an optional key-value parameters section, and then
+ * the message payload itself. Note: the IP address in the header may be either IPv4 (4 bytes) or IPv6 (16 bytes).
+ * The diagram below shows the IPv4 address for brevity.
+ *
+ * <pre>
+ * {@code
+ *            1 1 1 1 1 2 2 2 2 2 3 3 3 3 3 4 4 4 4 4 5 5 5 5 5 6 6
+ *  0 2 4 6 8 0 2 4 6 8 0 2 4 6 8 0 2 4 6 8 0 2 4 6 8 0 2 4 6 8 0 2
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * |                       PROTOCOL MAGIC                          |
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * |                         Message ID                            |
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * |                         Timestamp                             |
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * |  Addr len |           IP Address (IPv4)                       /
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * /           |                 Verb                              /
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * /           |            Parameters size                        /
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * /           |             Parameter data                        /
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * /                                                               |
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * |                        Payload size                           |
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * |                                                               /
+ * /                           Payload                             /
+ * /                                                               |
+ * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+ * }
+ * </pre>
+ *
+ * An individual parameter has a String key and a byte array value. The key is serialized with it's length,
+ * encoded as two bytes, followed by the UTF-8 byte encoding of the string (see {@link java.io.DataOutput#writeUTF(java.lang.String)}).
+ * The body is serialized with it's length, encoded as four bytes, followed by the bytes of the value.
+ *
+ * * @param <T> The type of the message payload.
+ */
 public class MessageOut<T>
 {
+    private static final int SERIALIZED_SIZE_VERSION_UNDEFINED = -1;
+
     public final InetAddress from;
     public final MessagingService.Verb verb;
     public final T payload;
     public final IVersionedSerializer<T> serializer;
     public final Map<String, byte[]> parameters;
-    private long payloadSize = -1;
-    private int payloadSizeVersion = -1;
+
+    /**
+     * Memoization of the serialized size of the just the payload.
+     */
+    private int payloadSerializedSize = -1;
+
+    /**
+     * Memoization of the serialized size of the entire message.
+     */
+    private int serializedSize = -1;
+
+    /**
+     * The internode protocol messaging version that was used to calculate the memoized serailized sizes.
+     */
+    private int serializedSizeVersion = SERIALIZED_SIZE_VERSION_UNDEFINED;
 
     // we do support messages that just consist of a verb
     public MessageOut(MessagingService.Verb verb)
@@ -115,14 +172,12 @@ public class MessageOut<T>
 
         if (payload != null)
         {
-            try(DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get())
-            {
-                serializer.serialize(payload, dob, version);
-
-                int size = dob.getLength();
-                out.writeInt(size);
-                out.write(dob.getData(), 0, size);
-            }
+            int payloadSize = payloadSerializedSize >= 0
+                              ? (int)payloadSerializedSize
+                              : (int) serializer.serializedSize(payload, version);
+
+            out.writeInt(payloadSize);
+            serializer.serialize(payload, out, version);
         }
         else
         {
@@ -130,9 +185,9 @@ public class MessageOut<T>
         }
     }
 
-    public int serializedSize(int version)
+    private Pair<Long, Long> calculateSerializedSize(int version)
     {
-        int size = CompactEndpointSerializationHelper.serializedSize(from);
+        long size = CompactEndpointSerializationHelper.serializedSize(from);
 
         size += TypeSizes.sizeof(verb.getId());
         size += TypeSizes.sizeof(parameters.size());
@@ -143,38 +198,43 @@ public class MessageOut<T>
             size += entry.getValue().length;
         }
 
-        long longSize = payloadSize(version);
-        assert longSize <= Integer.MAX_VALUE; // larger values are supported in sstables but not messages
-        size += TypeSizes.sizeof((int) longSize);
-        size += longSize;
-        return size;
+        long payloadSize = payload == null ? 0 : serializer.serializedSize(payload, version);
+        assert payloadSize <= Integer.MAX_VALUE; // larger values are supported in sstables but not messages
+        size += TypeSizes.sizeof((int) payloadSize);
+        size += payloadSize;
+        return Pair.create(size, payloadSize);
     }
 
     /**
-     * Calculate the size of the payload of this message for the specified protocol version
-     * and memoize the result for the specified protocol version. Memoization only covers the protocol
-     * version of the first invocation.
+     * Calculate the size of this message for the specified protocol version and memoize the result for the specified
+     * protocol version. Memoization only covers the protocol version of the first invocation.
      *
-     * It is not safe to call payloadSize concurrently from multiple threads unless it has already been invoked
+     * It is not safe to call this function concurrently from multiple threads unless it has already been invoked
      * once from a single thread and there is a happens before relationship between that invocation and other
-     * threads concurrently invoking payloadSize.
+     * threads concurrently invoking this function.
      *
      * For instance it would be safe to invokePayload size to make a decision in the thread that created the message
      * and then hand it off to other threads via a thread-safe queue, volatile write, or synchronized/ReentrantLock.
-     * @param version Protocol version to use when calculating payload size
-     * @return Size of the payload of this message in bytes
+     *
+     * @param version Protocol version to use when calculating size
+     * @return Size of this message in bytes, which will be less than or equal to {@link Integer#MAX_VALUE}
      */
-    public long payloadSize(int version)
+    public int serializedSize(int version)
     {
-        if (payloadSize == -1)
-        {
-            payloadSize = payload == null ? 0 : serializer.serializedSize(payload, version);
-            payloadSizeVersion = version;
-        }
-        else if (payloadSizeVersion != version)
+        if (serializedSize > 0 && serializedSizeVersion == version)
+            return serializedSize;
+
+        Pair<Long, Long> sizes = calculateSerializedSize(version);
+        if (sizes.left > Integer.MAX_VALUE)
+            throw new IllegalStateException("message size exceeds maximum allowed size: size = " + sizes.left);
+
+        if (serializedSizeVersion == SERIALIZED_SIZE_VERSION_UNDEFINED)
         {
-            return payload == null ? 0 : serializer.serializedSize(payload, version);
+            serializedSize = sizes.left.intValue();
+            payloadSerializedSize = sizes.right.intValue();
+            serializedSizeVersion = version;
         }
-        return payloadSize;
+
+        return sizes.left.intValue();
     }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[03/11] cassandra git commit: move streaming to use netty

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 3a95015..590ba5f 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -18,8 +18,6 @@
 package org.apache.cassandra.streaming;
 
 import java.io.*;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
 import java.util.Collection;
 import java.util.UUID;
 
@@ -30,8 +28,7 @@ import com.google.common.collect.UnmodifiableIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.ning.compress.lzf.LZFInputStream;
-
+import org.apache.cassandra.io.util.TrackedDataInputPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
@@ -42,9 +39,10 @@ import org.apache.cassandra.io.sstable.format.RangeAwareSSTableWriter;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.streaming.compress.StreamCompressionInputStream;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
+import org.apache.cassandra.streaming.messages.StreamMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.io.util.TrackedInputStream;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
@@ -88,12 +86,12 @@ public class StreamReader
     }
 
     /**
-     * @param channel where this reads data from
+     * @param inputPlus where this reads data from
      * @return SSTable transferred
      * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
      */
-    @SuppressWarnings("resource") // channel needs to remain open, streams on top of it can't be closed
-    public SSTableMultiWriter read(ReadableByteChannel channel) throws IOException
+    @SuppressWarnings("resource") // input needs to remain open, streams on top of it can't be closed
+    public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException
     {
         long totalSize = totalSize();
 
@@ -108,7 +106,8 @@ public class StreamReader
                      session.planId(), fileSeqNum, session.peer, repairedAt, totalSize, cfs.keyspace.getName(),
                      cfs.getTableName(), pendingRepair);
 
-        TrackedInputStream in = new TrackedInputStream(new LZFInputStream(Channels.newInputStream(channel)));
+
+        TrackedDataInputPlus in = new TrackedDataInputPlus(new StreamCompressionInputStream(inputPlus, StreamMessage.CURRENT_VERSION));
         StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata(), in, inputVersion, getHeader(cfs.metadata()));
         SSTableMultiWriter writer = null;
         try
@@ -179,10 +178,10 @@ public class StreamReader
         private Row staticRow;
         private IOException exception;
 
-        public StreamDeserializer(TableMetadata metadata, InputStream in, Version version, SerializationHeader header) throws IOException
+        public StreamDeserializer(TableMetadata metadata, DataInputPlus in, Version version, SerializationHeader header) throws IOException
         {
             this.metadata = metadata;
-            this.in = new DataInputPlus.DataInputStreamPlus(in);
+            this.in = in;
             this.helper = new SerializationHelper(metadata, version.correspondingMessagingVersion(), SerializationHelper.Flag.PRESERVE_SIZE);
             this.header = header;
         }
@@ -256,8 +255,8 @@ public class StreamReader
             // to what we do in hasNext)
             Unfiltered unfiltered = iterator.next();
             return metadata.isCounter() && unfiltered.kind() == Unfiltered.Kind.ROW
-                 ? maybeMarkLocalToBeCleared((Row) unfiltered)
-                 : unfiltered;
+                   ? maybeMarkLocalToBeCleared((Row) unfiltered)
+                   : unfiltered;
         }
 
         private Row maybeMarkLocalToBeCleared(Row row)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/StreamReceiveException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveException.java b/src/java/org/apache/cassandra/streaming/StreamReceiveException.java
new file mode 100644
index 0000000..54b365a
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveException.java
@@ -0,0 +1,36 @@
+/*
+ * 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 StreamReceiveException extends RuntimeException
+{
+    public final StreamSession session;
+
+    public StreamReceiveException(StreamSession session, String msg)
+    {
+        super(msg);
+        this.session = session;
+    }
+
+    public StreamReceiveException(StreamSession session, Throwable t)
+    {
+        super(t);
+        this.session = session;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index 925dc85..6aa70ad 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -104,6 +104,7 @@ public class StreamReceiveTask extends StreamTask
 
         remoteSSTablesReceived++;
         assert tableId.equals(sstable.getTableId());
+        logger.debug("recevied {} of {} total files", remoteSSTablesReceived, totalFiles);
 
         Collection<SSTableReader> finished = null;
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 67d7d0d..9371c65 100644
--- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
@@ -17,8 +17,9 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
 import java.util.*;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
@@ -27,7 +28,7 @@ import com.google.common.util.concurrent.Futures;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.net.IncomingStreamingConnection;
+import io.netty.channel.Channel;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -103,12 +104,10 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
                                                                     UUID planId,
                                                                     StreamOperation streamOperation,
                                                                     InetAddress from,
-                                                                    IncomingStreamingConnection connection,
-                                                                    boolean isForOutgoing,
-                                                                    int version,
+                                                                    Channel channel,
                                                                     boolean keepSSTableLevel,
                                                                     UUID pendingRepair,
-                                                                    PreviewKind previewKind) throws IOException
+                                                                    PreviewKind previewKind)
     {
         StreamResultFuture future = StreamManager.instance.getReceivingStream(planId);
         if (future == null)
@@ -119,7 +118,7 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
             future = new StreamResultFuture(planId, streamOperation, keepSSTableLevel, pendingRepair, previewKind);
             StreamManager.instance.registerReceiving(future);
         }
-        future.attachConnection(from, sessionIndex, connection, isForOutgoing, version);
+        future.attachConnection(from, sessionIndex, channel);
         logger.info("[Stream #{}, ID#{}] Received streaming plan for {}", planId, sessionIndex, streamOperation.getDescription());
         return future;
     }
@@ -131,11 +130,18 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
         return future;
     }
 
-    private void attachConnection(InetAddress from, int sessionIndex, IncomingStreamingConnection connection, boolean isForOutgoing, int version) throws IOException
+    public StreamCoordinator getCoordinator()
     {
-        StreamSession session = coordinator.getOrCreateSessionById(from, sessionIndex, connection.socket.getInetAddress());
+        return coordinator;
+    }
+
+    private void attachConnection(InetAddress from, int sessionIndex, Channel channel)
+    {
+        SocketAddress addr = channel.remoteAddress();
+        InetAddress connecting = (addr instanceof InetSocketAddress ? ((InetSocketAddress) addr).getAddress() : from);
+        StreamSession session = coordinator.getOrCreateSessionById(from, sessionIndex, connecting);
         session.init(this);
-        session.handler.initiateOnReceivingSide(connection, isForOutgoing, version);
+        session.attach(channel);
     }
 
     public void addEventListener(StreamEventHandler listener)
@@ -206,6 +212,7 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
 
     private synchronized void maybeComplete()
     {
+        logger.warn("[Stream #{}] maybeComplete", planId);
         if (!coordinator.hasActiveSessions())
         {
             StreamState finalState = getCurrentState();
@@ -221,4 +228,9 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
             }
         }
     }
+
+    StreamSession getSession(InetAddress peer, int sessionIndex)
+    {
+        return coordinator.getSessionById(peer, sessionIndex);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 5ca9938..0381416 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -17,9 +17,8 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.io.IOException;
 import java.net.InetAddress;
-import java.net.Socket;
+import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
 import java.util.*;
 import java.util.concurrent.*;
@@ -29,17 +28,21 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.*;
+import com.google.common.util.concurrent.Futures;
 
-import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.lifecycle.View;
+import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelId;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.PartitionPosition;
@@ -47,10 +50,12 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.*;
 import org.apache.cassandra.metrics.StreamingMetrics;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.async.NettyStreamingMessageSender;
 import org.apache.cassandra.streaming.messages.*;
-import org.apache.cassandra.utils.CassandraVersion;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Pair;
@@ -59,79 +64,80 @@ import org.apache.cassandra.utils.concurrent.Refs;
 
 /**
  * Handles the streaming a one or more section of one of more sstables to and from a specific
- * remote node.
+ * remote node. The sending side performs a block-level transfer of the source sstable, while the receiver
+ * must deserilaize that data stream into an partitions and rows, and then write that out as an sstable.
  *
- * Both this node and the remote one will create a similar symmetrical StreamSession. A streaming
+ * Both this node and the remote one will create a similar symmetrical {@link StreamSession}. A streaming
  * session has the following life-cycle:
  *
- * 1. Connections Initialization
+ * 1. Session Initialization
  *
- *   (a) A node (the initiator in the following) create a new StreamSession, initialize it (init())
- *       and then start it (start()). Start will create a {@link ConnectionHandler} that will create
- *       two connections to the remote node (the follower in the following) with whom to stream and send
- *       a StreamInit message. The first connection will be the incoming connection for the
- *       initiator, and the second connection will be the outgoing.
- *   (b) Upon reception of that StreamInit message, the follower creates its own StreamSession,
- *       initialize it if it still does not exist, and attach connecting socket to its ConnectionHandler
- *       according to StreamInit message's isForOutgoing flag.
- *   (d) When the both incoming and outgoing connections are established, StreamSession calls
- *       StreamSession#onInitializationComplete method to start the streaming prepare phase
- *       (StreamResultFuture.startStreaming()).
+ *   (a) A node (the initiator in the following) create a new {@link StreamSession},
+ *       initialize it {@link #init(StreamResultFuture)}, and then start it ({@link #start()}).
+ *       Starting a session causes a {@link StreamInitMessage} to be sent.
+ *   (b) Upon reception of that {@link StreamInitMessage}, the follower creates its own {@link StreamSession},
+ *       and initializes it if it still does not exist.
+ *   (c) After the initiator sends the {@link StreamInitMessage}, it invokes
+ *       {@link StreamSession#onInitializationComplete()} to start the streaming prepare phase.
  *
  * 2. Streaming preparation phase
  *
- *   (a) This phase is started when the initiator onInitializationComplete() method is called. This method sends a
- *       PrepareMessage that includes what files/sections this node will stream to the follower
- *       (stored in a StreamTransferTask, each column family has it's own transfer task) and what
- *       the follower needs to stream back (StreamReceiveTask, same as above). If the initiator has
- *       nothing to receive from the follower, it goes directly to its Streaming phase. Otherwise,
- *       it waits for the follower PrepareMessage.
- *   (b) Upon reception of the PrepareMessage, the follower records which files/sections it will receive
- *       and send back its own PrepareMessage with a summary of the files/sections that will be sent to
- *       the initiator (prepare()). After having sent that message, the follower goes to its Streamning
- *       phase.
- *   (c) When the initiator receives the follower PrepareMessage, it records which files/sections it will
- *       receive and then goes to his own Streaming phase.
+ *   (a) A {@link PrepareSynMessage} is sent that includes a) what files/sections this node will stream to the follower
+ *       (stored locally in a {@link StreamTransferTask}, one for each table) and b) what the follower needs to
+ *       stream back (stored locally in a {@link StreamReceiveTask}, one for each table).
+ *   (b) Upon reception of the {@link PrepareSynMessage}, the follower records which files/sections it will receive
+ *       and send back a {@link PrepareSynAckMessage}, which contains a summary of the files/sections that will be sent to
+ *       the initiator.
+ *   (c) When the initiator receives the {@link PrepareSynAckMessage}, it records which files/sections it will
+ *       receive, and then goes to it's Streaming phase (see next section). If the intiator is to receive files,
+ *       it sends a {@link PrepareAckMessage} to the follower to indicate that it can start streaming to the initiator.
+ *   (d) (Optional) If the follower receives a {@link PrepareAckMessage}, it enters it's Streaming phase.
  *
  * 3. Streaming phase
  *
- *   (a) The streaming phase is started by each node (the sender in the follower, but note that each side
- *       of the StreamSession may be sender for some of the files) involved by calling startStreamingFiles().
- *       This will sequentially send a FileMessage for each file of each SteamTransferTask. Each FileMessage
- *       consists of a FileMessageHeader that indicates which file is coming and then start streaming the
- *       content for that file (StreamWriter in FileMessage.serialize()). When a file is fully sent, the
- *       fileSent() method is called for that file. If all the files for a StreamTransferTask are sent
- *       (StreamTransferTask.complete()), the task is marked complete (taskCompleted()).
- *   (b) On the receiving side, a SSTable will be written for the incoming file (StreamReader in
- *       FileMessage.deserialize()) and once the FileMessage is fully received, the file will be marked as
- *       complete (received()). When all files for the StreamReceiveTask have been received, the sstables
- *       are added to the CFS (and 2ndary index are built, StreamReceiveTask.complete()) and the task
- *       is marked complete (taskCompleted())
+ *   (a) The streaming phase is started at each node by calling {@link StreamSession#startStreamingFiles(boolean)}.
+ *       This will send, sequentially on each outbound streaming connection (see {@link NettyStreamingMessageSender}),
+ *       an {@link OutgoingFileMessage} for each file in each of the {@link StreamTransferTask}.
+ *       Each {@link OutgoingFileMessage} consists of a {@link FileMessageHeader} that contains metadata about the file
+ *       being streamed, followed by the file content itself. Once all the files for a {@link StreamTransferTask} are sent,
+ *       the task is marked complete {@link StreamTransferTask#complete(int)}.
+ *   (b) On the receiving side, a SSTable will be written for the incoming file, and once the file is fully received,
+ *       the file will be marked as complete ({@link StreamReceiveTask#received(SSTableMultiWriter)}). When all files
+ *       for the {@link StreamReceiveTask} have been received, the sstables are added to the CFS (and 2ndary indexes/MV are built),
+ *        and the task is marked complete ({@link #taskCompleted(StreamReceiveTask)}).
  *   (b) If during the streaming of a particular file an error occurs on the receiving end of a stream
- *       (FileMessage.deserialize), the node will send a SessionFailedMessage to the sender and close the stream session.
- *   (c) When all transfer and receive tasks for a session are complete, the move to the Completion phase
- *       (maybeCompleted()).
+ *       (it may be either the initiator or the follower), the node will send a {@link SessionFailedMessage}
+ *       to the sender and close the stream session.
+ *   (c) When all transfer and receive tasks for a session are complete, the session moves to the Completion phase
+ *       ({@link #maybeCompleted()}).
  *
  * 4. Completion phase
  *
- *   (a) When a node has finished all transfer and receive task, it enter the completion phase (maybeCompleted()).
- *       If it had already received a CompleteMessage from the other side (it is in the WAIT_COMPLETE state), that
- *       session is done is is closed (closeSession()). Otherwise, the node switch to the WAIT_COMPLETE state and
- *       send a CompleteMessage to the other side.
+ *   (a) When a node enters the completion phase, it sends a {@link CompleteMessage} to the peer, and then enter the
+ *       {@link StreamSession.State#WAIT_COMPLETE} state. If it has already received a {@link CompleteMessage}
+ *       from the peer, session is complete and is then closed ({@link #closeSession(State)}). Otherwise, the node
+ *       switch to the {@link StreamSession.State#WAIT_COMPLETE} state and send a {@link CompleteMessage} to the other side.
+ *
+ * In brief, the message passing looks like this (I for initiator, F for follwer):
+ * (session init)
+ * I: StreamInitMessage
+ * (session prepare)
+ * I: PrepareSynMessage
+ * F: PrepareSynAckMessage
+ * I: PrepareAckMessage
+ * (stream - this can happen in both directions)
+ * I: OutgoingFileMessage
+ * F: ReceivedMessage
+ * (completion)
+ * I/F: CompleteMessage
+ *
+ * All messages which derive from {@link StreamMessage} are sent by the standard internode messaging
+ * (via {@link org.apache.cassandra.net.MessagingService}, while the actual files themselves are sent by a special
+ * "streaming" connection type. See {@link NettyStreamingMessageSender} for details. Because of the asynchronous
  */
 public class StreamSession implements IEndpointStateChangeSubscriber
 {
-
-    /**
-     * Version where keep-alive support was added
-     */
-    private static final CassandraVersion STREAM_KEEP_ALIVE_VERSION = new CassandraVersion("3.10");
     private static final Logger logger = LoggerFactory.getLogger(StreamSession.class);
-    private static final DebuggableScheduledThreadPoolExecutor keepAliveExecutor = new DebuggableScheduledThreadPoolExecutor("StreamKeepAliveExecutor");
-    static {
-        // Immediately remove keep-alive task when cancelled.
-        keepAliveExecutor.setRemoveOnCancelPolicy(true);
-    }
 
     /**
      * Streaming endpoint.
@@ -139,7 +145,9 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      * Each {@code StreamSession} is identified by this InetAddress which is broadcast address of the node streaming.
      */
     public final InetAddress peer;
+
     private final int index;
+
     /** Actual connecting address. Can be the same as {@linkplain #peer}. */
     public final InetAddress connecting;
 
@@ -154,20 +162,18 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     // data receivers, filled after receiving prepare message
     private final Map<TableId, StreamReceiveTask> receivers = new ConcurrentHashMap<>();
     private final StreamingMetrics metrics;
-    /* can be null when session is created in remote */
-    private final StreamConnectionFactory factory;
 
-    public final Map<String, Set<Range<Token>>> transferredRangesPerKeyspace = new HashMap<>();
+    final Map<String, Set<Range<Token>>> transferredRangesPerKeyspace = new HashMap<>();
 
-    public final ConnectionHandler handler;
+    private final NettyStreamingMessageSender messageSender;
+    private final ConcurrentMap<ChannelId, Channel> incomingChannels = new ConcurrentHashMap<>();
 
-    private AtomicBoolean isAborted = new AtomicBoolean(false);
+    private final AtomicBoolean isAborted = new AtomicBoolean(false);
     private final boolean keepSSTableLevel;
-    private ScheduledFuture<?> keepAliveFuture = null;
     private final UUID pendingRepair;
     private final PreviewKind previewKind;
 
-    public static enum State
+    public enum State
     {
         INITIALIZED,
         PREPARING,
@@ -184,17 +190,16 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      * Create new streaming session with the peer.
      *  @param peer Address of streaming peer
      * @param connecting Actual connecting address
-     * @param factory is used for establishing connection
      */
     public StreamSession(InetAddress peer, InetAddress connecting, StreamConnectionFactory factory, int index, boolean keepSSTableLevel, UUID pendingRepair, PreviewKind previewKind)
     {
         this.peer = peer;
         this.connecting = connecting;
         this.index = index;
-        this.factory = factory;
-        this.handler = new ConnectionHandler(this, isKeepAliveSupported()?
-                                                   (int)TimeUnit.SECONDS.toMillis(2 * DatabaseDescriptor.getStreamingKeepAlivePeriod()) :
-                                                   DatabaseDescriptor.getStreamingSocketTimeout(), previewKind.isPreview());
+
+        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.stream(new InetSocketAddress(FBUtilities.getBroadcastAddress(), 0),
+                                                                              new InetSocketAddress(connecting, MessagingService.portFor(connecting)));
+        this.messageSender = new NettyStreamingMessageSender(this, id, factory, StreamMessage.CURRENT_VERSION, previewKind.isPreview());
         this.metrics = StreamingMetrics.get(connecting);
         this.keepSSTableLevel = keepSSTableLevel;
         this.pendingRepair = pendingRepair;
@@ -242,12 +247,6 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         return receivers.get(tableId).getTransaction();
     }
 
-    private boolean isKeepAliveSupported()
-    {
-        CassandraVersion peerVersion = Gossiper.instance.getReleaseVersion(peer);
-        return peerVersion != null && peerVersion.compareTo(STREAM_KEEP_ALIVE_VERSION) >= 0;
-    }
-
     /**
      * Bind this session to report to specific {@link StreamResultFuture} and
      * perform pre-streaming initialization.
@@ -258,13 +257,18 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     {
         this.streamResult = streamResult;
         StreamHook.instance.reportStreamFuture(this, streamResult);
+    }
 
-        if (isKeepAliveSupported())
-            scheduleKeepAliveTask();
-        else
-            logger.debug("Peer {} does not support keep-alive.", peer);
+    public boolean attach(Channel channel)
+    {
+        if (!messageSender.hasControlChannel())
+            messageSender.injectControlMessageChannel(channel);
+        return incomingChannels.putIfAbsent(channel.id(), channel) == null;
     }
 
+    /**
+     * invoked by the node that begins the stream session (it may be sending files, receiving files, or both)
+     */
     public void start()
     {
         if (requests.isEmpty() && transfers.isEmpty())
@@ -279,7 +283,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
             logger.info("[Stream #{}] Starting streaming to {}{}", planId(),
                                                                    peer,
                                                                    peer.equals(connecting) ? "" : " through " + connecting);
-            handler.initiate();
+            messageSender.initialize();
             onInitializationComplete();
         }
         catch (Exception e)
@@ -289,12 +293,6 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         }
     }
 
-    public Socket createConnection() throws IOException
-    {
-        assert factory != null;
-        return factory.createConnection(connecting);
-    }
-
     /**
      * Request data fetch task to this session.
      *
@@ -317,7 +315,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      * @param columnFamilies Transfer ColumnFamilies
      * @param flushTables flush tables?
      */
-    public synchronized void addTransferRanges(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies, boolean flushTables)
+    synchronized void addTransferRanges(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies, boolean flushTables)
     {
         failIfFinished();
         Collection<ColumnFamilyStore> stores = getColumnFamilyStores(keyspace, columnFamilies);
@@ -428,7 +426,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         }
     }
 
-    public synchronized void addTransferFiles(Collection<SSTableStreamingSections> sstableDetails)
+    synchronized void addTransferFiles(Collection<SSTableStreamingSections> sstableDetails)
     {
         failIfFinished();
         Iterator<SSTableStreamingSections> iter = sstableDetails.iterator();
@@ -472,31 +470,37 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         }
     }
 
-    private synchronized void closeSession(State finalState)
+    private synchronized Future closeSession(State finalState)
     {
+        Future abortedTasksFuture = null;
         if (isAborted.compareAndSet(false, true))
         {
             state(finalState);
 
+            // ensure aborting the tasks do not happen on the network IO thread (read: netty event loop)
+            // as we don't want any blocking disk IO to stop the network thread
             if (finalState == State.FAILED)
-            {
-                for (StreamTask task : Iterables.concat(receivers.values(), transfers.values()))
-                    task.abort();
-            }
+                abortedTasksFuture = ScheduledExecutors.nonPeriodicTasks.submit(this::abortTasks);
 
-            if (keepAliveFuture != null)
-            {
-                logger.debug("[Stream #{}] Finishing keep-alive task.", planId());
-                keepAliveFuture.cancel(false);
-                keepAliveFuture = null;
-            }
-
-            // Note that we shouldn't block on this close because this method is called on the handler
-            // incoming thread (so we would deadlock).
-            handler.close();
+            incomingChannels.values().stream().map(channel -> channel.close());
+            messageSender.close();
 
             streamResult.handleSessionComplete(this);
         }
+        return abortedTasksFuture != null ? abortedTasksFuture : Futures.immediateFuture(null);
+    }
+
+    private void abortTasks()
+    {
+        try
+        {
+            receivers.values().forEach(StreamReceiveTask::abort);
+            transfers.values().forEach(StreamTransferTask::abort);
+        }
+        catch (Exception e)
+        {
+            logger.warn("failed to abort some streaming tasks", e);
+        }
     }
 
     /**
@@ -517,6 +521,11 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         return state;
     }
 
+    public NettyStreamingMessageSender getMessageSender()
+    {
+        return messageSender;
+    }
+
     /**
      * Return if this session completed successfully.
      *
@@ -531,27 +540,37 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     {
         switch (message.type)
         {
-            case PREPARE:
-                PrepareMessage msg = (PrepareMessage) message;
+            case STREAM_INIT:
+                // nop
+                break;
+            case PREPARE_SYN:
+                PrepareSynMessage msg = (PrepareSynMessage) message;
                 prepare(msg.requests, msg.summaries);
                 break;
-
+            case PREPARE_SYNACK:
+                prepareSynAck((PrepareSynAckMessage) message);
+                break;
+            case PREPARE_ACK:
+                prepareAck((PrepareAckMessage) message);
+                break;
             case FILE:
                 receive((IncomingFileMessage) message);
                 break;
-
             case RECEIVED:
                 ReceivedMessage received = (ReceivedMessage) message;
                 received(received.tableId, received.sequenceNumber);
                 break;
-
             case COMPLETE:
                 complete();
                 break;
-
+            case KEEP_ALIVE:
+                // NOP - we only send/receive the KEEP_ALIVE to force the TCP connection to remain open
+                break;
             case SESSION_FAILED:
                 sessionFailed();
                 break;
+            default:
+                throw new AssertionError("unhandled StreamMessage type: " + message.getClass().getName());
         }
     }
 
@@ -562,55 +581,43 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     {
         // send prepare message
         state(State.PREPARING);
-        PrepareMessage prepare = new PrepareMessage();
+        PrepareSynMessage prepare = new PrepareSynMessage();
         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())
-            startStreamingFiles();
+        messageSender.sendMessage(prepare);
     }
 
-    /**l
+    /**
      * Call back for handling exception during streaming.
-     *
-     * @param e thrown exception
      */
-    public void onError(Throwable e)
+    public Future onError(Throwable e)
     {
         logError(e);
         // send session failure message
-        if (handler.isOutgoingConnected())
-            handler.sendMessage(new SessionFailedMessage());
+        if (messageSender.connected())
+            messageSender.sendMessage(new SessionFailedMessage());
         // fail session
-        closeSession(State.FAILED);
+        return closeSession(State.FAILED);
     }
 
     private void logError(Throwable e)
     {
         if (e instanceof SocketTimeoutException)
         {
-            if (isKeepAliveSupported())
-                logger.error("[Stream #{}] Did not receive response from peer {}{} for {} secs. Is peer down? " +
-                             "If not, maybe try increasing streaming_keep_alive_period_in_secs.", planId(),
-                             peer.getHostAddress(),
-                             peer.equals(connecting) ? "" : " through " + connecting.getHostAddress(),
-                             2 * DatabaseDescriptor.getStreamingKeepAlivePeriod(),
-                             e);
-            else
-                logger.error("[Stream #{}] Streaming socket timed out. This means the session peer stopped responding or " +
-                             "is still processing received data. If there is no sign of failure in the other end or a very " +
-                             "dense table is being transferred you may want to increase streaming_socket_timeout_in_ms " +
-                             "property. Current value is {}ms.", planId(), DatabaseDescriptor.getStreamingSocketTimeout(), e);
+            logger.error("[Stream #{}] Did not receive response from peer {}{} for {} secs. Is peer down? " +
+                         "If not, maybe try increasing streaming_keep_alive_period_in_secs.", planId(),
+                         peer.getHostAddress(),
+                         peer.equals(connecting) ? "" : " through " + connecting.getHostAddress(),
+                         2 * DatabaseDescriptor.getStreamingKeepAlivePeriod(),
+                         e);
         }
         else
         {
             logger.error("[Stream #{}] Streaming error occurred on session with peer {}{}", planId(),
-                                                                                            peer.getHostAddress(),
-                                                                                            peer.equals(connecting) ? "" : " through " + connecting.getHostAddress(),
-                                                                                            e);
+                         peer.getHostAddress(),
+                         peer.equals(connecting) ? "" : " through " + connecting.getHostAddress(),
+                         e);
         }
     }
 
@@ -621,29 +628,55 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     {
         // prepare tasks
         state(State.PREPARING);
+        ScheduledExecutors.nonPeriodicTasks.execute(() -> prepareAsync(requests, summaries));
+    }
+
+    /**
+     * Finish preparing the session. This method is blocking (memtables are flushed in {@link #addTransferRanges}),
+     * so the logic should not execute on the main IO thread (read: netty event loop).
+     */
+    private void prepareAsync(Collection<StreamRequest> requests, Collection<StreamSummary> summaries)
+    {
+
         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();
+        PrepareSynAckMessage prepareSynAck = new PrepareSynAckMessage();
+        if (!peer.equals(FBUtilities.getBroadcastAddress()))
             for (StreamTransferTask task : transfers.values())
-                prepare.summaries.add(task.getSummary());
-            handler.sendMessage(prepare);
+                prepareSynAck.summaries.add(task.getSummary());
+        messageSender.sendMessage(prepareSynAck);
+
+
+        streamResult.handleSessionPrepared(this);
+        maybeCompleted();
+    }
+
+    private void prepareSynAck(PrepareSynAckMessage msg)
+    {
+        if (!msg.summaries.isEmpty())
+        {
+            for (StreamSummary summary : msg.summaries)
+                prepareReceiving(summary);
+
+            // only send the (final) ACK if we are expecting the peer to send this node (the initiator) some files
+            messageSender.sendMessage(new PrepareAckMessage());
         }
 
         if (isPreview())
-        {
             completePreview();
-            return;
-        }
+        else
+            startStreamingFiles(true);
+    }
 
-        // if there are files to stream
-        if (!maybeCompleted())
-            startStreamingFiles();
+    private void prepareAck(PrepareAckMessage msg)
+    {
+        if (isPreview())
+            completePreview();
+        else
+            startStreamingFiles(true);
     }
 
     /**
@@ -665,7 +698,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     }
 
     /**
-     * Call back after receiving FileMessageHeader.
+     * Call back after receiving a streamed file.
      *
      * @param message received file
      */
@@ -680,7 +713,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         StreamingMetrics.totalIncomingBytes.inc(headerSize);
         metrics.incomingBytes.inc(headerSize);
         // send back file received message
-        handler.sendMessage(new ReceivedMessage(message.header.tableId, message.header.sequenceNumber));
+        messageSender.sendMessage(new ReceivedMessage(message.header.tableId, message.header.sequenceNumber));
         receivers.get(message.header.tableId).received(message.sstable);
     }
 
@@ -700,11 +733,12 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      */
     public synchronized void complete()
     {
+        logger.debug("handling Complete message, state = {}, completeSent = {}", state, completeSent);
         if (state == State.WAIT_COMPLETE)
         {
             if (!completeSent)
             {
-                handler.sendMessage(new CompleteMessage());
+                messageSender.sendMessage(new CompleteMessage());
                 completeSent = true;
             }
             closeSession(State.COMPLETE);
@@ -712,17 +746,6 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         else
         {
             state(State.WAIT_COMPLETE);
-            handler.closeIncoming();
-        }
-    }
-
-    private synchronized void scheduleKeepAliveTask()
-    {
-        if (keepAliveFuture == null)
-        {
-            int keepAlivePeriod = DatabaseDescriptor.getStreamingKeepAlivePeriod();
-            logger.debug("[Stream #{}] Scheduling keep-alive task with {}s period.", planId(), keepAlivePeriod);
-            keepAliveFuture = keepAliveExecutor.scheduleAtFixedRate(new KeepAliveTask(), 0, keepAlivePeriod, TimeUnit.SECONDS);
         }
     }
 
@@ -804,7 +827,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
             {
                 if (!completeSent)
                 {
-                    handler.sendMessage(new CompleteMessage());
+                    messageSender.sendMessage(new CompleteMessage());
                     completeSent = true;
                 }
                 closeSession(State.COMPLETE);
@@ -812,10 +835,9 @@ public class StreamSession implements IEndpointStateChangeSubscriber
             else
             {
                 // notify peer that this session is completed
-                handler.sendMessage(new CompleteMessage());
+                messageSender.sendMessage(new CompleteMessage());
                 completeSent = true;
                 state(State.WAIT_COMPLETE);
-                handler.closeOutgoing();
             }
         }
         return completed;
@@ -840,46 +862,30 @@ public class StreamSession implements IEndpointStateChangeSubscriber
             receivers.put(summary.tableId, new StreamReceiveTask(this, summary.tableId, summary.files, summary.totalSize));
     }
 
-    private void startStreamingFiles()
+    private void startStreamingFiles(boolean notifyPrepared)
     {
-        streamResult.handleSessionPrepared(this);
+        if (notifyPrepared)
+            streamResult.handleSessionPrepared(this);
 
         state(State.STREAMING);
 
         for (StreamTransferTask task : transfers.values())
         {
             Collection<OutgoingFileMessage> messages = task.getFileMessages();
-            if (messages.size() > 0)
-                handler.sendMessages(messages);
-            else
-                taskCompleted(task); // there is no file to send
-        }
-    }
-
-    class KeepAliveTask implements Runnable
-    {
-        private KeepAliveMessage last = null;
-
-        public void run()
-        {
-            //to avoid jamming the message queue, we only send if the last one was sent
-            if (last == null || last.wasSent())
+            if (!messages.isEmpty())
             {
-                logger.trace("[Stream #{}] Sending keep-alive to {}.", planId(), peer);
-                last = new KeepAliveMessage();
-                try
+                for (OutgoingFileMessage ofm : messages)
                 {
-                    handler.sendMessage(last);
-                }
-                catch (RuntimeException e) //connection handler is closed
-                {
-                    logger.debug("[Stream #{}] Could not send keep-alive message (perhaps stream session is finished?).", planId(), e);
+                    // pass the session planId/index to the OFM (which is only set at init(), after the transfers have already been created)
+                    ofm.header.addSessionInfo(this);
+                    messageSender.sendMessage(ofm);
                 }
             }
             else
             {
-                logger.trace("[Stream #{}] Skip sending keep-alive to {} (previous was not yet sent).", planId(), peer);
+                taskCompleted(task); // there are no files to send
             }
         }
+        maybeCompleted();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 748da8b..5e21712 100644
--- a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
@@ -20,11 +20,15 @@ package org.apache.cassandra.streaming;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.schema.TableId;
@@ -37,6 +41,7 @@ import org.apache.cassandra.utils.concurrent.Ref;
  */
 public class StreamTransferTask extends StreamTask
 {
+    private static final Logger logger = LoggerFactory.getLogger(StreamTransferTask.class);
     private static final ScheduledExecutorService timeoutExecutor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("StreamingTransferTaskTimeouts"));
 
     private final AtomicInteger sequenceNumber = new AtomicInteger(0);
@@ -56,10 +61,10 @@ public class StreamTransferTask extends StreamTask
     public synchronized void addTransferFile(Ref<SSTableReader> ref, long estimatedKeys, List<Pair<Long, Long>> sections)
     {
         assert ref.get() != null && tableId.equals(ref.get().metadata().id);
-        OutgoingFileMessage message = new OutgoingFileMessage(ref, sequenceNumber.getAndIncrement(), estimatedKeys, sections, session.keepSSTableLevel());
+        OutgoingFileMessage message = new OutgoingFileMessage(ref, session, sequenceNumber.getAndIncrement(), estimatedKeys, sections, session.keepSSTableLevel());
         message = StreamHook.instance.reportOutgoingFile(session, ref.get(), message);
         files.put(message.header.sequenceNumber, message);
-        totalSize += message.header.size();
+                totalSize += message.header.size();
     }
 
     /**
@@ -80,6 +85,7 @@ public class StreamTransferTask extends StreamTask
             if (file != null)
                 file.complete();
 
+            logger.debug("recevied sequenceNumber {}, remaining files {}", sequenceNumber, files.keySet());
             signalComplete = files.isEmpty();
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/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
index 6c86c8b..81b3d8a 100644
--- a/src/java/org/apache/cassandra/streaming/StreamWriter.java
+++ b/src/java/org/apache/cassandra/streaming/StreamWriter.java
@@ -19,21 +19,21 @@ package org.apache.cassandra.streaming;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.util.Collection;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.ning.compress.lzf.LZFOutputStream;
-
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
 import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter;
+import org.apache.cassandra.streaming.compress.ByteBufCompressionDataOutputStreamPlus;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
@@ -51,11 +51,6 @@ public class StreamWriter
     protected final StreamRateLimiter limiter;
     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;
@@ -78,45 +73,48 @@ public class StreamWriter
         logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(),
                      sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize);
 
-        try(RandomAccessReader file = sstable.openDataReader();
+        try(ChannelProxy proxy = sstable.getDataChannel().sharedCopy();
             ChecksumValidator validator = new File(sstable.descriptor.filenameFor(Component.CRC)).exists()
                                           ? DataIntegrityMetadata.checksumValidator(sstable.descriptor)
-                                          : null;)
+                                          : null)
         {
-            transferBuffer = validator == null ? new byte[DEFAULT_CHUNK_SIZE] : new byte[validator.chunkSize];
+            int bufferSize = validator == null ? DEFAULT_CHUNK_SIZE: validator.chunkSize;
 
             // setting up data compression stream
-            compressedOutput = new LZFOutputStream(output);
             long progress = 0L;
 
-            // stream each of the required sections of the file
-            for (Pair<Long, Long> section : sections)
+            try (DataOutputStreamPlus compressedOutput = new ByteBufCompressionDataOutputStreamPlus(output, limiter))
             {
-                long start = validator == null ? section.left : validator.chunkStart(section.left);
-                int readOffset = (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 bytesRead = 0;
-                while (bytesRead < length)
+                // stream each of the required sections of the file
+                for (Pair<Long, Long> section : sections)
                 {
-                    long lastBytesRead = write(file, validator, readOffset, length, bytesRead);
-                    bytesRead += lastBytesRead;
-                    progress += (lastBytesRead - readOffset);
-                    session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize);
-                    readOffset = 0;
+                    long start = validator == null ? section.left : validator.chunkStart(section.left);
+                    // if the transfer does not start on the valididator's chunk boundary, this is the number of bytes to offset by
+                    int transferOffset = (int) (section.left - start);
+                    if (validator != null)
+                        validator.seek(start);
+
+                    // length of the section to read
+                    long length = section.right - start;
+                    // tracks write progress
+                    long bytesRead = 0;
+                    while (bytesRead < length)
+                    {
+                        int toTransfer = (int) Math.min(bufferSize, length - bytesRead);
+                        long lastBytesRead = write(proxy, validator, compressedOutput, start, transferOffset, toTransfer, bufferSize);
+                        start += lastBytesRead;
+                        bytesRead += lastBytesRead;
+                        progress += (lastBytesRead - transferOffset);
+                        session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize);
+                        transferOffset = 0;
+                    }
+
+                    // make sure that current section is sent
+                    output.flush();
                 }
-
-                // make sure that current section is sent
-                compressedOutput.flush();
+                logger.debug("[Stream #{}] Finished streaming file {} to {}, bytesTransferred = {}, totalSize = {}",
+                             session.planId(), sstable.getFilename(), session.peer, FBUtilities.prettyPrintMemory(progress), FBUtilities.prettyPrintMemory(totalSize));
             }
-            logger.debug("[Stream #{}] Finished streaming file {} to {}, bytesTransferred = {}, totalSize = {}",
-                         session.planId(), sstable.getFilename(), session.peer, FBUtilities.prettyPrintMemory(progress), FBUtilities.prettyPrintMemory(totalSize));
         }
     }
 
@@ -131,27 +129,44 @@ public class StreamWriter
     /**
      * Sequentially read bytes from the file and write them to the output stream
      *
-     * @param reader The file reader to read from
+     * @param proxy 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 read from {@code reader}
-     * @param bytesTransferred Number of bytes already read out of {@code length}
+     * @param start The readd offset from the beginning of the {@code proxy} file.
+     * @param transferOffset number of bytes to skip transfer, but include for validation.
+     * @param toTransfer The number of bytes to be transferred.
      *
-     * @return Number of bytes read
+     * @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
+    protected long write(ChannelProxy proxy, ChecksumValidator validator, DataOutputStreamPlus output, long start, int transferOffset, int toTransfer, int bufferSize) throws IOException
     {
-        int toTransfer = (int) Math.min(transferBuffer.length, length - bytesTransferred);
-        int minReadable = (int) Math.min(transferBuffer.length, reader.length() - reader.getFilePointer());
+        // the count of bytes to read off disk
+        int minReadable = (int) Math.min(bufferSize, proxy.size() - start);
 
-        reader.readFully(transferBuffer, 0, minReadable);
-        if (validator != null)
-            validator.validate(transferBuffer, 0, minReadable);
+        // this buffer will hold the data from disk. as it will be compressed on the fly by
+        // ByteBufCompressionDataOutputStreamPlus.write(ByteBuffer), we can release this buffer as soon as we can.
+        ByteBuffer buffer = ByteBuffer.allocateDirect(minReadable);
+        try
+        {
+            int readCount = proxy.read(buffer, start);
+            assert readCount == minReadable : String.format("could not read required number of bytes from file to be streamed: read %d bytes, wanted %d bytes", readCount, minReadable);
+            buffer.flip();
 
-        limiter.acquire(toTransfer - start);
-        compressedOutput.write(transferBuffer, start, (toTransfer - start));
+            if (validator != null)
+            {
+                validator.validate(buffer);
+                buffer.flip();
+            }
+
+            buffer.position(transferOffset);
+            buffer.limit(transferOffset + (toTransfer - transferOffset));
+            output.write(buffer);
+        }
+        finally
+        {
+            FileUtils.clean(buffer);
+        }
 
         return toTransfer;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
new file mode 100644
index 0000000..9562981
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
@@ -0,0 +1,34 @@
+/*
+ * 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 org.apache.cassandra.streaming.messages.StreamMessage;
+
+public interface StreamingMessageSender
+{
+    void initialize() throws IOException;
+
+    void sendMessage(StreamMessage message) throws IOException;
+
+    boolean connected();
+
+    void close();
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
new file mode 100644
index 0000000..f872005
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
@@ -0,0 +1,508 @@
+/*
+ * 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.async;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelPipeline;
+import io.netty.util.AttributeKey;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.GenericFutureListener;
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.util.DataOutputBufferFixed;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus;
+import org.apache.cassandra.net.async.NettyFactory;
+import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
+import org.apache.cassandra.streaming.StreamConnectionFactory;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamingMessageSender;
+import org.apache.cassandra.streaming.messages.IncomingFileMessage;
+import org.apache.cassandra.streaming.messages.KeepAliveMessage;
+import org.apache.cassandra.streaming.messages.OutgoingFileMessage;
+import org.apache.cassandra.streaming.messages.StreamInitMessage;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * Responsible for sending {@link StreamMessage}s to a given peer. We manage an array of netty {@link Channel}s
+ * for sending {@link OutgoingFileMessage} instances; all other {@link StreamMessage} types are sent via
+ * a special control channel. The reason for this is to treat those messages carefully and not let them get stuck
+ * behind a file transfer.
+ *
+ * One of the challenges when sending files is we might need to delay shipping the file if:
+ *
+ * - we've exceeded our network I/O use due to rate limiting (at the cassandra level)
+ * - the receiver isn't keeping up, which causes the local TCP socket buffer to not empty, which causes epoll writes to not
+ * move any bytes to the socket, which causes buffers to stick around in user-land (a/k/a cassandra) memory.
+ *
+ * When those conditions occur, it's easy enough to reschedule processing the file once the resources pick up
+ * (we acquire the permits from the rate limiter, or the socket drains). However, we need to ensure that
+ * no other messages are submitted to the same channel while the current file is still being processed.
+ */
+public class NettyStreamingMessageSender implements StreamingMessageSender
+{
+    private static final Logger logger = LoggerFactory.getLogger(NettyStreamingMessageSender.class);
+
+    private static final int DEFAULT_MAX_PARALLEL_TRANSFERS = FBUtilities.getAvailableProcessors();
+    private static final int MAX_PARALLEL_TRANSFERS = Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "streaming.session.parallelTransfers", Integer.toString(DEFAULT_MAX_PARALLEL_TRANSFERS)));
+
+    // a simple mechansim for allowing a degree of fairnes across multiple sessions
+    private static final Semaphore fileTransferSemaphore = new Semaphore(DEFAULT_MAX_PARALLEL_TRANSFERS, true);
+
+    private final StreamSession session;
+    private final boolean isPreview;
+    private final int protocolVersion;
+    private final OutboundConnectionIdentifier connectionId;
+    private final StreamConnectionFactory factory;
+
+    private volatile boolean closed;
+
+    /**
+     * A special {@link Channel} for sending non-file streaming messages, basically anything that isn't an
+     * {@link OutgoingFileMessage} (or an {@link IncomingFileMessage}, but a node doesn't send that, it's only received).
+     */
+    private Channel controlMessageChannel;
+
+    // note: this really doesn't need to be a LBQ, just something that's thread safe
+    private final Collection<ScheduledFuture<?>> channelKeepAlives = new LinkedBlockingQueue<>();
+
+    private final ThreadPoolExecutor fileTransferExecutor;
+
+    /**
+     * A {@link ThreadLocal} used by the threads in {@link #fileTransferExecutor} to stash references to constructed
+     * and connected {@link Channel}s.
+     */
+    private final ConcurrentMap<Thread, Channel> threadLocalChannel = new ConcurrentHashMap<>();
+
+    /**
+     * A netty channel attribute used to indicate if a channel is currently transferring a file. This is primarily used
+     * to indicate to the {@link KeepAliveTask} if it is safe to send a {@link KeepAliveMessage}, as sending the
+     * (application level) keep-alive in the middle of streaming a file would be bad news.
+     */
+    @VisibleForTesting
+    static final AttributeKey<Boolean> TRANSFERRING_FILE_ATTR = AttributeKey.valueOf("transferringFile");
+
+    public NettyStreamingMessageSender(StreamSession session, OutboundConnectionIdentifier connectionId, StreamConnectionFactory factory, int protocolVersion, boolean isPreview)
+    {
+        this.session = session;
+        this.protocolVersion = protocolVersion;
+        this.connectionId = connectionId;
+        this.factory = factory;
+        this.isPreview = isPreview;
+
+        String name = session.peer.toString().replace(':', '.');
+        fileTransferExecutor = new DebuggableThreadPoolExecutor(1, MAX_PARALLEL_TRANSFERS, 1L, TimeUnit.SECONDS, new LinkedBlockingQueue<>(),
+                                                                new NamedThreadFactory("NettyStreaming-Outbound-" + name));
+        fileTransferExecutor.allowCoreThreadTimeOut(true);
+    }
+
+    @Override
+    public void initialize() throws IOException
+    {
+        StreamInitMessage message = new StreamInitMessage(FBUtilities.getBroadcastAddress(),
+                                                          session.sessionIndex(),
+                                                          session.planId(),
+                                                          session.streamOperation(),
+                                                          session.keepSSTableLevel(),
+                                                          session.getPendingRepair(),
+                                                          session.getPreviewKind());
+        sendMessage(message);
+    }
+
+    public boolean hasControlChannel()
+    {
+        return controlMessageChannel != null;
+    }
+
+    public void injectControlMessageChannel(Channel channel)
+    {
+        this.controlMessageChannel = channel;
+        channel.attr(TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
+        scheduleKeepAliveTask(channel);
+    }
+
+    private void setupControlMessageChannel() throws IOException
+    {
+        if (controlMessageChannel == null)
+        {
+            controlMessageChannel = createChannel();
+            scheduleKeepAliveTask(controlMessageChannel);
+        }
+    }
+
+    private void scheduleKeepAliveTask(Channel channel)
+    {
+        int keepAlivePeriod = DatabaseDescriptor.getStreamingKeepAlivePeriod();
+        logger.debug("{} Scheduling keep-alive task with {}s period.", createLogTag(session, channel), keepAlivePeriod);
+
+        KeepAliveTask task = new KeepAliveTask(channel, session);
+        ScheduledFuture<?> scheduledFuture = channel.eventLoop().scheduleAtFixedRate(task, 0, keepAlivePeriod, TimeUnit.SECONDS);
+        channelKeepAlives.add(scheduledFuture);
+        task.future = scheduledFuture;
+    }
+    
+    private Channel createChannel() throws IOException
+    {
+        Channel channel = factory.createConnection(connectionId, protocolVersion);
+        ChannelPipeline pipeline = channel.pipeline();
+        pipeline.addLast(NettyFactory.instance.streamingGroup, NettyFactory.INBOUND_STREAM_HANDLER_NAME, new StreamingInboundHandler(connectionId.remoteAddress(), protocolVersion, session));
+        channel.attr(TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
+        return channel;
+    }
+
+     static String createLogTag(StreamSession session, Channel channel)
+    {
+        StringBuilder sb = new StringBuilder(64);
+        sb.append("[Stream");
+
+        if (session != null)
+                sb.append(" #").append(session.planId());
+
+        if (channel != null)
+                sb.append(" channel: ").append(channel.id());
+
+        sb.append(']');
+        return sb.toString();
+    }
+
+    @Override
+    public void sendMessage(StreamMessage message)
+    {
+        if (closed)
+            throw new RuntimeException("stream has been closed, cannot send " + message);
+
+        if (message instanceof OutgoingFileMessage)
+        {
+            if (isPreview)
+                throw new RuntimeException("Cannot send file messages for preview streaming sessions");
+            logger.debug("{} Sending {}", createLogTag(session, null), message);
+            fileTransferExecutor.submit(new FileStreamTask((OutgoingFileMessage)message));
+            return;
+        }
+
+        try
+        {
+            setupControlMessageChannel();
+            sendControlMessage(controlMessageChannel, message, future -> onControlMessageComplete(future, message));
+        }
+        catch (Exception e)
+        {
+            close();
+            session.onError(e);
+        }
+    }
+
+    private void sendControlMessage(Channel channel, StreamMessage message, GenericFutureListener listener) throws IOException
+    {
+        logger.debug("{} Sending {}", createLogTag(session, channel), message);
+
+        // we anticipate that the control messages are rather small, so allocating a ByteBuf shouldn't  blow out of memory.
+        long messageSize = StreamMessage.serializedSize(message, protocolVersion);
+        if (messageSize > 1 << 30)
+        {
+            throw new IllegalStateException(String.format("%s something is seriously wrong with the calculated stream control message's size: %d bytes, type is %s",
+                                                          createLogTag(session, channel), messageSize, message.type));
+        }
+
+        // as control messages are (expected to be) small, we can simply allocate a ByteBuf here, wrap it, and send via the channel
+        ByteBuf buf = channel.alloc().directBuffer((int) messageSize, (int) messageSize);
+        ByteBuffer nioBuf = buf.nioBuffer(0, (int) messageSize);
+        @SuppressWarnings("resource")
+        DataOutputBufferFixed out = new DataOutputBufferFixed(nioBuf);
+        StreamMessage.serialize(message, out, protocolVersion, session);
+        assert nioBuf.position() == nioBuf.limit();
+        buf.writerIndex(nioBuf.position());
+
+        ChannelFuture channelFuture = channel.writeAndFlush(buf);
+        channelFuture.addListener(future -> listener.operationComplete(future));
+    }
+
+    /**
+     * Decides what to do after a {@link StreamMessage} is processed.
+     *
+     * Note: this is called from the netty event loop.
+     *
+     * @return null if the message was processed sucessfully; else, a {@link java.util.concurrent.Future} to indicate
+     * the status of aborting any remaining tasks in the session.
+     */
+    java.util.concurrent.Future onControlMessageComplete(Future<?> future, StreamMessage msg)
+    {
+        ChannelFuture channelFuture = (ChannelFuture)future;
+        Throwable cause = future.cause();
+        if (cause == null)
+            return null;
+
+        Channel channel = channelFuture.channel();
+        logger.error("{} failed to send a stream message/file to peer {}: msg = {}",
+                     createLogTag(session, channel), connectionId, msg, future.cause());
+
+        // StreamSession will invoke close(), but we have to mark this sender as closed so the session doesn't try
+        // to send any failure messages
+        return session.onError(cause);
+    }
+
+    class FileStreamTask implements Runnable
+    {
+        /**
+         * Time interval, in minutes, to wait between logging a message indicating that we're waiting on a semaphore
+         * permit to become available.
+         */
+        private static final int SEMAPHORE_UNAVAILABLE_LOG_INTERVAL = 3;
+
+        /**
+         * Even though we expect only an {@link OutgoingFileMessage} at runtime, the type here is {@link StreamMessage}
+         * to facilitate simpler testing.
+         */
+        private final StreamMessage msg;
+
+        FileStreamTask(OutgoingFileMessage ofm)
+        {
+            this.msg = ofm;
+        }
+
+        /**
+         * For testing purposes
+         */
+        FileStreamTask(StreamMessage msg)
+        {
+            this.msg = msg;
+        }
+
+        @Override
+        public void run()
+        {
+            if (!acquirePermit(SEMAPHORE_UNAVAILABLE_LOG_INTERVAL))
+                return;
+
+            try
+            {
+                Channel channel = getOrCreateChannel();
+                if (!channel.attr(TRANSFERRING_FILE_ATTR).compareAndSet(false, true))
+                    throw new IllegalStateException("channel's transferring state is currently set to true. refusing to start new stream");
+
+                // close the DataOutputStreamPlus as we're done with it - but don't close the channel
+                try (DataOutputStreamPlus outPlus = ByteBufDataOutputStreamPlus.create(session, channel, 1 << 16))
+                {
+                    StreamMessage.serialize(msg, outPlus, protocolVersion, session);
+                    channel.flush();
+                }
+                finally
+                {
+                    channel.attr(TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
+                }
+            }
+            catch (Exception e)
+            {
+                session.onError(e);
+            }
+            finally
+            {
+                fileTransferSemaphore.release();
+            }
+        }
+
+        boolean acquirePermit(int logInterval)
+        {
+            long logIntervalNanos = TimeUnit.MINUTES.toNanos(logInterval);
+            long timeOfLastLogging = System.nanoTime();
+            while (true)
+            {
+                if (closed)
+                    return false;
+                try
+                {
+                    if (fileTransferSemaphore.tryAcquire(1, TimeUnit.SECONDS))
+                        return true;
+
+                    // log a helpful message to operators in case they are wondering why a given session might not be making progress.
+                    long now = System.nanoTime();
+                    if (now - timeOfLastLogging > logIntervalNanos)
+                    {
+                        timeOfLastLogging = now;
+                        OutgoingFileMessage ofm = (OutgoingFileMessage)msg;
+                        logger.info("{} waiting to acquire a permit to begin streaming file {}. This message logs every {} minutes",
+                                    createLogTag(session, null), ofm.getFilename(), logInterval);
+                    }
+                }
+                catch (InterruptedException ie)
+                {
+                    //ignore
+                }
+            }
+        }
+
+        private Channel getOrCreateChannel()
+        {
+            Thread currentThread = Thread.currentThread();
+            try
+            {
+                Channel channel = threadLocalChannel.get(currentThread);
+                if (channel != null)
+                    return channel;
+
+                channel = createChannel();
+                threadLocalChannel.put(currentThread, channel);
+                return channel;
+            }
+            catch (Exception e)
+            {
+                throw new IOError(e);
+            }
+        }
+
+        /**
+         * For testing purposes
+         */
+        void injectChannel(Channel channel)
+        {
+            Thread currentThread = Thread.currentThread();
+            if (threadLocalChannel.get(currentThread) != null)
+                throw new IllegalStateException("previous channel already set");
+
+            threadLocalChannel.put(currentThread, channel);
+        }
+
+        /**
+         * For testing purposes
+         */
+        void unsetChannel()
+        {
+            threadLocalChannel.remove(Thread.currentThread());
+        }
+    }
+
+    /**
+     * Periodically sends the {@link KeepAliveMessage}.
+     *
+     * NOTE: this task, and the callback function {@link #keepAliveListener(Future)} is executed in the netty event loop.
+     */
+    class KeepAliveTask implements Runnable
+    {
+        private final Channel channel;
+        private final StreamSession session;
+
+        /**
+         * A reference to the scheduled task for this instance so that it may be cancelled.
+         */
+        ScheduledFuture<?> future;
+
+        KeepAliveTask(Channel channel, StreamSession session)
+        {
+            this.channel = channel;
+            this.session = session;
+        }
+
+        public void run()
+        {
+            // if the channel has been closed, cancel the scheduled task and return
+            if (!channel.isOpen() || closed)
+            {
+                future.cancel(false);
+                return;
+            }
+
+            // if the channel is currently processing streaming, skip this execution. As this task executes
+            // on the event loop, even if there is a race with a FileStreamTask which changes the channel attribute
+            // after we check it, the FileStreamTask cannot send out any bytes as this KeepAliveTask is executing
+            // on the event loop (and FileStreamTask publishes it's buffer to the channel, consumed after we're done here).
+            if (channel.attr(TRANSFERRING_FILE_ATTR).get())
+                return;
+
+            try
+            {
+                logger.trace("{} Sending keep-alive to {}.", createLogTag(session, channel), session.peer);
+                sendControlMessage(channel, new KeepAliveMessage(), this::keepAliveListener);
+            }
+            catch (IOException ioe)
+            {
+                future.cancel(false);
+            }
+        }
+
+        private void keepAliveListener(Future<? super Void> future)
+        {
+            if (future.isSuccess() || future.isCancelled())
+                return;
+
+            logger.debug("{} Could not send keep-alive message (perhaps stream session is finished?).",
+                         createLogTag(session, channel), future.cause());
+        }
+    }
+
+    /**
+     * For testing purposes only.
+     */
+    void setClosed()
+    {
+        closed = true;
+    }
+
+    void setControlMessageChannel(Channel channel)
+    {
+        controlMessageChannel = channel;
+    }
+
+    int semaphoreAvailablePermits()
+    {
+        return fileTransferSemaphore.availablePermits();
+    }
+
+    @Override
+    public boolean connected()
+    {
+        return !closed;
+    }
+
+    @Override
+    public void close()
+    {
+        closed = true;
+        logger.debug("{} Closing stream connection channels on {}", createLogTag(session, null), connectionId);
+        channelKeepAlives.stream().map(scheduledFuture -> scheduledFuture.cancel(false));
+        channelKeepAlives.clear();
+
+        threadLocalChannel.values().stream().map(channel -> channel.close());
+        threadLocalChannel.clear();
+        fileTransferExecutor.shutdownNow();
+
+        if (controlMessageChannel != null)
+            controlMessageChannel.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc92db2b/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java b/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java
new file mode 100644
index 0000000..ca15b78
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java
@@ -0,0 +1,133 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4FastDecompressor;
+import org.apache.cassandra.io.util.DataInputPlus;
+
+/**
+ * A serialiazer for stream compressed files (see package-level documentation). Much like a typical compressed
+ * output stream, this class operates on buffers or chunks of the data at a a time. The format for each compressed
+ * chunk is as follows:
+ *
+ * - int - compressed payload length
+ * - int - uncompressed payload length
+ * - bytes - compressed payload
+ */
+public class StreamCompressionSerializer
+{
+    private final ByteBufAllocator allocator;
+
+    public StreamCompressionSerializer(ByteBufAllocator allocator)
+    {
+        this.allocator = allocator;
+    }
+
+    /**
+     * Length of heaer data, which includes compressed length, uncompressed length.
+     */
+    private static final int HEADER_LENGTH = 8;
+
+    /**
+     * @return A buffer with decompressed data.
+     */
+    public ByteBuf serialize(LZ4Compressor compressor, ByteBuffer in, int version)
+    {
+        final int uncompressedLength = in.remaining();
+        int maxLength = compressor.maxCompressedLength(uncompressedLength);
+        ByteBuf out = allocator.directBuffer(maxLength);
+        try
+        {
+            ByteBuffer compressedNioBuffer = out.nioBuffer(HEADER_LENGTH, maxLength - HEADER_LENGTH);
+            compressor.compress(in, compressedNioBuffer);
+            final int compressedLength = compressedNioBuffer.position();
+            out.setInt(0, compressedLength);
+            out.setInt(4, uncompressedLength);
+            out.writerIndex(HEADER_LENGTH + compressedLength);
+        }
+        catch (Exception e)
+        {
+            if (out != null)
+                out.release();
+        }
+        return out;
+    }
+
+    /**
+     * @return A buffer with decompressed data.
+     */
+    public ByteBuf deserialize(LZ4FastDecompressor decompressor, DataInputPlus in, int version) throws IOException
+    {
+        final int compressedLength = in.readInt();
+        final int uncompressedLength = in.readInt();
+
+        // there's no guarantee the next compressed block is contained within one buffer in the input,
+        // so hence we need a 'staging' buffer to get all the bytes into one contiguous buffer for the decompressor
+        ByteBuf compressed = null;
+        ByteBuf uncompressed = null;
+        try
+        {
+            final ByteBuffer compressedNioBuffer;
+
+            // ReadableByteChannel allows us to keep the bytes off-heap because we pass a ByteBuffer to RBC.read(BB),
+            // DataInputPlus.read() takes a byte array (thus, an on-heap array).
+            if (in instanceof ReadableByteChannel)
+            {
+                compressed = allocator.directBuffer(compressedLength);
+                compressedNioBuffer = compressed.nioBuffer(0, compressedLength);
+                int readLength = ((ReadableByteChannel) in).read(compressedNioBuffer);
+                assert readLength == compressedNioBuffer.position();
+                compressedNioBuffer.flip();
+            }
+            else
+            {
+                byte[] compressedBytes = new byte[compressedLength];
+                in.readFully(compressedBytes);
+                compressedNioBuffer = ByteBuffer.wrap(compressedBytes);
+            }
+
+            uncompressed = allocator.directBuffer(uncompressedLength);
+            ByteBuffer uncompressedNioBuffer = uncompressed.nioBuffer(0, uncompressedLength);
+            decompressor.decompress(compressedNioBuffer, uncompressedNioBuffer);
+            uncompressed.writerIndex(uncompressedLength);
+            return uncompressed;
+        }
+        catch (Exception e)
+        {
+            if (uncompressed != null)
+                uncompressed.release();
+
+            if (e instanceof IOException)
+                throw e;
+            throw new IOException(e);
+        }
+        finally
+        {
+            if (compressed != null)
+                compressed.release();
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[07/11] cassandra git commit: switch internode messaging to netty

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/security/SSLFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java
index 33c1ad6..3c1293f 100644
--- a/src/java/org/apache/cassandra/security/SSLFactory.java
+++ b/src/java/org/apache/cassandra/security/SSLFactory.java
@@ -17,63 +17,67 @@
  */
 package org.apache.cassandra.security;
 
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.io.InputStream;
+
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.security.KeyStore;
 import java.security.cert.X509Certificate;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.Enumeration;
 import java.util.List;
-
+import java.util.concurrent.atomic.AtomicReference;
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLParameters;
-import javax.net.ssl.SSLServerSocket;
 import javax.net.ssl.SSLSocket;
 import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
 
-import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.io.util.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.handler.ssl.ClientAuth;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+import io.netty.handler.ssl.SupportedCipherSuiteFilter;
+import io.netty.util.ReferenceCountUtil;
+import org.apache.cassandra.config.EncryptionOptions;
 
 /**
- * A Factory for providing and setting up Client and Server SSL wrapped
- * Socket and ServerSocket
+ * A Factory for providing and setting up client {@link SSLSocket}s. Also provides
+ * methods for creating both JSSE {@link SSLContext} instances as well as netty {@link SslContext} instances.
+ *
+ * Netty {@link SslContext} instances are expensive to create (as well as to destroy) and consume a lof of resources
+ * (especially direct memory), but instances can be reused across connections (assuming the SSL params are the same).
+ * Hence we cache created instances in {@link #clientSslContext} and {@link #serverSslContext}.
  */
 public final class SSLFactory
 {
     private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
-    private static boolean checkedExpiry = false;
 
-    public static SSLServerSocket getServerSocket(EncryptionOptions options, InetAddress address, int port) throws IOException
-    {
-        SSLContext ctx = createSSLContext(options, true);
-        SSLServerSocket serverSocket = (SSLServerSocket)ctx.getServerSocketFactory().createServerSocket();
-        try
-        {
-            serverSocket.setReuseAddress(true);
-            prepareSocket(serverSocket, options);
-            serverSocket.bind(new InetSocketAddress(address, port), 500);
-            return serverSocket;
-        }
-        catch (IllegalArgumentException | SecurityException | IOException e)
-        {
-            serverSocket.close();
-            throw e;
-        }
-    }
+    @VisibleForTesting
+    static volatile boolean checkedExpiry = false;
+
+    /**
+     * A cached reference of the {@link SslContext} for client-facing connections.
+     */
+    private static final AtomicReference<SslContext> clientSslContext = new AtomicReference<>();
+
+    /**
+     * A cached reference of the {@link SslContext} for peer-to-peer, internode messaging connections.
+     */
+    private static final AtomicReference<SslContext> serverSslContext = new AtomicReference<>();
 
     /** Create a socket and connect */
     public static SSLSocket getSocket(EncryptionOptions options, InetAddress address, int port, InetAddress localAddress, int localPort) throws IOException
@@ -109,37 +113,6 @@ public final class SSLFactory
         }
     }
 
-    /** Just create a socket */
-    public static SSLSocket getSocket(EncryptionOptions options) throws IOException
-    {
-        SSLContext ctx = createSSLContext(options, true);
-        SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket();
-        try
-        {
-            prepareSocket(socket, options);
-            return socket;
-        }
-        catch (IllegalArgumentException e)
-        {
-            socket.close();
-            throw e;
-        }
-    }
-
-    /** Sets relevant socket options specified in encryption settings */
-    private static void prepareSocket(SSLServerSocket serverSocket, EncryptionOptions options)
-    {
-        String[] suites = filterCipherSuites(serverSocket.getSupportedCipherSuites(), options.cipher_suites);
-        if(options.require_endpoint_verification)
-        {
-            SSLParameters sslParameters = serverSocket.getSSLParameters();
-            sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
-            serverSocket.setSSLParameters(sslParameters);
-        }
-        serverSocket.setEnabledCipherSuites(suites);
-        serverSocket.setNeedClientAuth(options.require_client_auth);
-    }
-
     /** Sets relevant socket options specified in encryption settings */
     private static void prepareSocket(SSLSocket socket, EncryptionOptions options)
     {
@@ -153,28 +126,50 @@ public final class SSLFactory
         socket.setEnabledCipherSuites(suites);
     }
 
+    /**
+     * Create a JSSE {@link SSLContext}.
+     */
     @SuppressWarnings("resource")
     public static SSLContext createSSLContext(EncryptionOptions options, boolean buildTruststore) throws IOException
     {
-        InputStream tsf = null;
-        InputStream ksf = null;
-        SSLContext ctx;
+        TrustManager[] trustManagers = null;
+        if (buildTruststore)
+            trustManagers = buildTrustManagerFactory(options).getTrustManagers();
+
+        KeyManagerFactory kmf = buildKeyManagerFactory(options);
+
         try
         {
-            ctx = SSLContext.getInstance(options.protocol);
-            TrustManager[] trustManagers = null;
+            SSLContext ctx = SSLContext.getInstance(options.protocol);
+            ctx.init(kmf.getKeyManagers(), trustManagers, null);
+            return ctx;
+        }
+        catch (Exception e)
+        {
+            throw new IOException("Error creating/initializing the SSL Context", e);
+        }
+    }
 
-            if(buildTruststore)
-            {
-                tsf = Files.newInputStream(Paths.get(options.truststore));
-                TrustManagerFactory tmf = TrustManagerFactory.getInstance(options.algorithm);
-                KeyStore ts = KeyStore.getInstance(options.store_type);
-                ts.load(tsf, options.truststore_password.toCharArray());
-                tmf.init(ts);
-                trustManagers = tmf.getTrustManagers();
-            }
+    static TrustManagerFactory buildTrustManagerFactory(EncryptionOptions options) throws IOException
+    {
+        try (InputStream tsf = Files.newInputStream(Paths.get(options.truststore)))
+        {
+            TrustManagerFactory tmf = TrustManagerFactory.getInstance(options.algorithm);
+            KeyStore ts = KeyStore.getInstance(options.store_type);
+            ts.load(tsf, options.truststore_password.toCharArray());
+            tmf.init(ts);
+            return tmf;
+        }
+        catch (Exception e)
+        {
+            throw new IOException("failed to build trust manager store for secure connections", e);
+        }
+    }
 
-            ksf = Files.newInputStream(Paths.get((options.keystore)));
+    static KeyManagerFactory buildKeyManagerFactory(EncryptionOptions options) throws IOException
+    {
+        try (InputStream ksf = Files.newInputStream(Paths.get(options.keystore)))
+        {
             KeyManagerFactory kmf = KeyManagerFactory.getInstance(options.algorithm);
             KeyStore ks = KeyStore.getInstance(options.store_type);
             ks.load(ksf, options.keystore_password.toCharArray());
@@ -193,20 +188,12 @@ public final class SSLFactory
                 checkedExpiry = true;
             }
             kmf.init(ks, options.keystore_password.toCharArray());
-
-            ctx.init(kmf.getKeyManagers(), trustManagers, null);
-
+            return kmf;
         }
         catch (Exception e)
         {
-            throw new IOException("Error creating the initializing the SSL Context", e);
+            throw new IOException("failed to build trust manager store for secure connections", e);
         }
-        finally
-        {
-            FileUtils.closeQuietly(tsf);
-            FileUtils.closeQuietly(ksf);
-        }
-        return ctx;
     }
 
     public static String[] filterCipherSuites(String[] supported, String[] desired)
@@ -223,4 +210,65 @@ public final class SSLFactory
         }
         return ret;
     }
+
+    /**
+     * get a netty {@link SslContext} instance
+     */
+    public static SslContext getSslContext(EncryptionOptions options, boolean buildTruststore, boolean forServer) throws IOException
+    {
+        return getSslContext(options, buildTruststore, forServer, OpenSsl.isAvailable());
+    }
+
+    /**
+     * Get a netty {@link SslContext} instance.
+     */
+    @VisibleForTesting
+    static SslContext getSslContext(EncryptionOptions options, boolean buildTruststore, boolean forServer, boolean useOpenSsl) throws IOException
+    {
+        if (forServer && serverSslContext.get() != null)
+            return serverSslContext.get();
+        if (!forServer && clientSslContext.get() != null)
+            return clientSslContext.get();
+
+        /*
+            There is a case where the netty/openssl combo might not support using KeyManagerFactory. specifically,
+            I've seen this with the netty-tcnative dynamic openssl implementation. using the netty-tcnative static-boringssl
+            works fine with KeyManagerFactory. If we want to support all of the netty-tcnative options, we would need
+            to fall back to passing in a file reference for both a x509 and PKCS#8 private key file in PEM format (see
+            {@link SslContextBuilder#forServer(File, File, String)}). However, we are not supporting that now to keep
+            the config/yaml API simple.
+         */
+        KeyManagerFactory kmf = null;
+        if (forServer || options.require_client_auth)
+            kmf = buildKeyManagerFactory(options);
+
+        SslContextBuilder builder;
+        if (forServer)
+        {
+            builder = SslContextBuilder.forServer(kmf);
+            builder.clientAuth(options.require_client_auth ? ClientAuth.REQUIRE : ClientAuth.NONE);
+        }
+        else
+        {
+            builder = SslContextBuilder.forClient().keyManager(kmf);
+        }
+
+        builder.sslProvider(useOpenSsl ? SslProvider.OPENSSL : SslProvider.JDK);
+
+        // only set the cipher suites if the opertor has explicity configured values for it; else, use the default
+        // for each ssl implemention (jdk or openssl)
+        if (options.cipher_suites != null && options.cipher_suites.length > 0)
+            builder.ciphers(Arrays.asList(options.cipher_suites), SupportedCipherSuiteFilter.INSTANCE);
+
+        if (buildTruststore)
+            builder.trustManager(buildTrustManagerFactory(options));
+
+        SslContext ctx = builder.build();
+        AtomicReference<SslContext> ref = forServer ? serverSslContext : clientSslContext;
+        if (ref.compareAndSet(null, ctx))
+            return ctx;
+
+        ReferenceCountUtil.release(ctx);
+        return ref.get();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
index a2ad66c..d88d63c 100644
--- a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
+++ b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
@@ -19,13 +19,18 @@ package org.apache.cassandra.streaming;
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.Socket;
+import java.nio.channels.SocketChannel;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.net.OutboundTcpConnectionPool;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.security.SSLFactory;
+import org.apache.cassandra.utils.FBUtilities;
 
 public class DefaultConnectionFactory implements StreamConnectionFactory
 {
@@ -47,20 +52,15 @@ public class DefaultConnectionFactory implements StreamConnectionFactory
         int attempts = 0;
         while (true)
         {
-            Socket socket = null;
             try
             {
-                socket = OutboundTcpConnectionPool.newSocket(peer);
+                Socket socket = newSocket(peer);
                 socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
                 socket.setKeepAlive(true);
                 return socket;
             }
             catch (IOException e)
             {
-                if (socket != null)
-                {
-                    socket.close();
-                }
                 if (++attempts >= MAX_CONNECT_ATTEMPTS)
                     throw e;
 
@@ -77,4 +77,21 @@ public class DefaultConnectionFactory implements StreamConnectionFactory
             }
         }
     }
+
+    // TODO this is deliberately copied from (the now former) OutboundTcpConnectionPool, for CASSANDRA-8457.
+    // to be replaced in CASSANDRA-12229 (make streaming use 8457)
+    public static Socket newSocket(InetAddress endpoint) throws IOException
+    {
+        // zero means 'bind on any available port.'
+        if (MessagingService.isEncryptedConnection(endpoint))
+        {
+            return SSLFactory.getSocket(DatabaseDescriptor.getServerEncryptionOptions(), endpoint, DatabaseDescriptor.getSSLStoragePort());
+        }
+        else
+        {
+            SocketChannel channel = SocketChannel.open();
+            channel.connect(new InetSocketAddress(endpoint, DatabaseDescriptor.getStoragePort()));
+            return channel.socket();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index 33e1967..4cdddba 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -316,8 +316,7 @@ public abstract class Tracing implements ExecutorLocal<TraceState>
     }
 
     /**
-     * Called from {@link org.apache.cassandra.net.OutboundTcpConnection} for non-local traces (traces
-     * that are not initiated by local node == coordinator).
+     * Called for non-local traces (traces that are not initiated by local node == coordinator).
      */
     public abstract void trace(ByteBuffer sessionId, String message, int ttl);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/tracing/TracingImpl.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/TracingImpl.java b/src/java/org/apache/cassandra/tracing/TracingImpl.java
index d774abb..789216e 100644
--- a/src/java/org/apache/cassandra/tracing/TracingImpl.java
+++ b/src/java/org/apache/cassandra/tracing/TracingImpl.java
@@ -99,8 +99,7 @@ class TracingImpl extends Tracing
     }
 
     /**
-     * Called from {@link org.apache.cassandra.net.OutboundTcpConnection} for non-local traces (traces
-     * that are not initiated by local node == coordinator).
+     * Called for non-local traces (traces that are not initiated by local node == coordinator).
      */
     public void trace(final ByteBuffer sessionId, final String message, final int ttl)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/transport/Message.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index 881ee81..1afe910 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -610,7 +610,9 @@ public abstract class Message
                 message = "Unexpected exception during request; channel = <unprintable>";
             }
 
-            if (!alwaysLogAtError && exception instanceof IOException)
+            // netty wraps SSL errors in a CodecExcpetion
+            boolean isIOException = exception instanceof IOException || (exception.getCause() instanceof IOException);
+            if (!alwaysLogAtError && isIOException)
             {
                 if (ioExceptionsAtDebugLevel.contains(exception.getMessage()))
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/transport/Server.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java
index 28f99e8..9408a3a 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -32,6 +32,7 @@ import org.slf4j.LoggerFactory;
 
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
 import io.netty.channel.*;
 import io.netty.channel.epoll.EpollEventLoopGroup;
 import io.netty.channel.epoll.EpollServerSocketChannel;
@@ -40,6 +41,7 @@ import io.netty.channel.group.DefaultChannelGroup;
 import io.netty.channel.nio.NioEventLoopGroup;
 import io.netty.channel.socket.nio.NioServerSocketChannel;
 import io.netty.handler.codec.ByteToMessageDecoder;
+import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslHandler;
 import io.netty.util.Version;
 import io.netty.util.concurrent.EventExecutor;
@@ -343,31 +345,18 @@ public class Server implements CassandraDaemon.Server
 
     protected abstract static class AbstractSecureIntializer extends Initializer
     {
-        private final SSLContext sslContext;
         private final EncryptionOptions encryptionOptions;
 
         protected AbstractSecureIntializer(Server server, EncryptionOptions encryptionOptions)
         {
             super(server);
             this.encryptionOptions = encryptionOptions;
-            try
-            {
-                this.sslContext = SSLFactory.createSSLContext(encryptionOptions, encryptionOptions.require_client_auth);
-            }
-            catch (IOException e)
-            {
-                throw new RuntimeException("Failed to setup secure pipeline", e);
-            }
         }
 
-        protected final SslHandler createSslHandler()
+        protected final SslHandler createSslHandler(ByteBufAllocator allocator) throws IOException
         {
-            SSLEngine sslEngine = sslContext.createSSLEngine();
-            sslEngine.setUseClientMode(false);
-            String[] suites = SSLFactory.filterCipherSuites(sslEngine.getSupportedCipherSuites(), encryptionOptions.cipher_suites);
-            sslEngine.setEnabledCipherSuites(suites);
-            sslEngine.setNeedClientAuth(encryptionOptions.require_client_auth);
-            return new SslHandler(sslEngine);
+            SslContext sslContext = SSLFactory.getSslContext(encryptionOptions, encryptionOptions.require_client_auth, true);
+            return sslContext.newHandler(allocator);
         }
     }
 
@@ -396,7 +385,7 @@ public class Server implements CassandraDaemon.Server
                     {
                         // Connection uses SSL/TLS, replace the detection handler with a SslHandler and so use
                         // encryption.
-                        SslHandler sslHandler = createSslHandler();
+                        SslHandler sslHandler = createSslHandler(channel.alloc());
                         channelHandlerContext.pipeline().replace(this, "ssl", sslHandler);
                     }
                     else
@@ -419,7 +408,7 @@ public class Server implements CassandraDaemon.Server
 
         protected void initChannel(Channel channel) throws Exception
         {
-            SslHandler sslHandler = createSslHandler();
+            SslHandler sslHandler = createSslHandler(channel.alloc());
             super.initChannel(channel);
             channel.pipeline().addFirst("ssl", sslHandler);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/transport/SimpleClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java b/src/java/org/apache/cassandra/transport/SimpleClient.java
index 13cd9bd..c72d6e9 100644
--- a/src/java/org/apache/cassandra/transport/SimpleClient.java
+++ b/src/java/org/apache/cassandra/transport/SimpleClient.java
@@ -28,8 +28,6 @@ import java.util.Map;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.SynchronousQueue;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLEngine;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,6 +38,7 @@ import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.SimpleChannelInboundHandler;
 import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.handler.ssl.SslContext;
 import io.netty.util.internal.logging.InternalLoggerFactory;
 import io.netty.util.internal.logging.Slf4JLoggerFactory;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -58,7 +57,6 @@ import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelPipeline;
-import io.netty.handler.ssl.SslHandler;
 import static org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
 
 public class SimpleClient implements Closeable
@@ -287,21 +285,11 @@ public class SimpleClient implements Closeable
 
     private class SecureInitializer extends Initializer
     {
-        private final SSLContext sslContext;
-
-        public SecureInitializer() throws IOException
-        {
-            this.sslContext = SSLFactory.createSSLContext(encryptionOptions, true);
-        }
-
         protected void initChannel(Channel channel) throws Exception
         {
             super.initChannel(channel);
-            SSLEngine sslEngine = sslContext.createSSLEngine();
-            sslEngine.setUseClientMode(true);
-            String[] suites = SSLFactory.filterCipherSuites(sslEngine.getSupportedCipherSuites(), encryptionOptions.cipher_suites);
-            sslEngine.setEnabledCipherSuites(suites);
-            channel.pipeline().addFirst("ssl", new SslHandler(sslEngine));
+            SslContext sslContext = SSLFactory.getSslContext(encryptionOptions, encryptionOptions.require_client_auth, true);
+            channel.pipeline().addFirst("ssl", sslContext.newHandler(channel.alloc()));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/utils/CoalescingStrategies.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CoalescingStrategies.java b/src/java/org/apache/cassandra/utils/CoalescingStrategies.java
index 9f3b118..6ff91e3 100644
--- a/src/java/org/apache/cassandra/utils/CoalescingStrategies.java
+++ b/src/java/org/apache/cassandra/utils/CoalescingStrategies.java
@@ -17,13 +17,10 @@
  */
 package org.apache.cassandra.utils;
 
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.util.FileUtils;
-
 import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.RandomAccessFile;
@@ -32,19 +29,17 @@ import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel.MapMode;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.LockSupport;
 import java.util.Locale;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 
+/**
+ * Groups strategies to coalesce messages.
+ */
 public class CoalescingStrategies
 {
-    static protected final Logger logger = LoggerFactory.getLogger(CoalescingStrategies.class);
-
     /*
      * Log debug information at info level about what the average is and when coalescing is enabled/disabled
      */
@@ -54,6 +49,8 @@ public class CoalescingStrategies
     private static final String DEBUG_COALESCING_PATH_PROPERTY = Config.PROPERTY_PREFIX + "coalescing_debug_path";
     private static final String DEBUG_COALESCING_PATH = System.getProperty(DEBUG_COALESCING_PATH_PROPERTY, "/tmp/coleascing_debug");
 
+    public enum Strategy { MOVINGAVERAGE, FIXED, TIMEHORIZON, DISABLED }
+
     static
     {
         if (DEBUG_COALESCING)
@@ -68,98 +65,83 @@ public class CoalescingStrategies
         }
     }
 
-    @VisibleForTesting
-    interface Clock
-    {
-        long nanoTime();
-    }
-
-    @VisibleForTesting
-    static Clock CLOCK = new Clock()
-    {
-        public long nanoTime()
-        {
-            return System.nanoTime();
-        }
-    };
-
     public static interface Coalescable
     {
         long timestampNanos();
     }
 
     @VisibleForTesting
-    static void parkLoop(long nanos)
-    {
-        long now = System.nanoTime();
-        final long timer = now + nanos;
-        // We shouldn't loop if it's within a few % of the target sleep time if on a second iteration.
-        // See CASSANDRA-8692.
-        final long limit = timer - nanos / 16;
-        do
-        {
-            LockSupport.parkNanos(timer - now);
-            now = System.nanoTime();
-        }
-        while (now < limit);
-    }
-
-    private static boolean maybeSleep(int messages, long averageGap, long maxCoalesceWindow, Parker parker)
+    static long determineCoalescingTime(long averageGap, long maxCoalesceWindow)
     {
-        // Do not sleep if there are still items in the backlog (CASSANDRA-13090).
-        if (messages >= DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages())
-            return false;
+        // Don't bother waiting at all if we're unlikely to get any new message within our max window
+        if (averageGap > maxCoalesceWindow)
+            return -1;
 
-        // only sleep if we can expect to double the number of messages we're sending in the time interval
-        long sleep = messages * averageGap;
-        if (sleep <= 0 || sleep > maxCoalesceWindow)
-            return false;
+        // avoid the degenerate case of zero (very unlikely, but let's be safe)
+        if (averageGap <= 0)
+            return maxCoalesceWindow;
 
         // assume we receive as many messages as we expect; apply the same logic to the future batch:
         // expect twice as many messages to consider sleeping for "another" interval; this basically translates
-        // to doubling our sleep period until we exceed our max sleep window
+        // to doubling our sleep period until we exceed our max sleep window.
+        long sleep = averageGap;
         while (sleep * 2 < maxCoalesceWindow)
             sleep *= 2;
-        parker.park(sleep);
-        return true;
+        return sleep;
     }
 
-    public static abstract class CoalescingStrategy
+    /**
+     * A coalescing strategy, that decides when to coalesce messages.
+     * <p>
+     * The general principle is that, when asked, the strategy returns the time delay we want to wait for more messages
+     * to arrive before sending so message can be coalesced. For that, the strategy must be fed new messages through
+     * the {@link #newArrival(Coalescable)} method (the only assumption we make on messages is that they have an associated
+     * timestamp). The strategy can then be queried for the time to wait for coalescing through
+     * {@link #currentCoalescingTimeNanos()}.
+     * <p>
+     * Note that it is expected that a call {@link #currentCoalescingTimeNanos()} will come just after a call to
+     * {@link #newArrival(Coalescable))}, as the intent of the value returned by the former method is "Given a new message, how much
+     * time should I wait for more messages to arrive and be coalesced with that message". But both calls are separated
+     * as one may not want to call {@link #currentCoalescingTimeNanos()} after every call to {@link #newArrival(Coalescable)}
+     * and we thus save processing. How arrivals influence the coalescing time is however entirely up to the strategy and some
+     * strategy may ignore arrivals completely and return a constant coalescing time.
+     */
+    public interface CoalescingStrategy
+    {
+        /**
+         * Inform the strategy of a new message to consider.
+         *
+         * @param message the message to consider.
+         */
+        void newArrival(Coalescable message);
+
+        /**
+         * The current time to wait for the purpose of coalescing messages.
+         *
+         * @return the coalescing time. A negative value can be returned if no coalescing should be done (which can be a
+         * transient thing).
+         */
+        long currentCoalescingTimeNanos();
+    }
+
+    public static abstract class AbstractCoalescingStrategy implements CoalescingStrategy
     {
-        protected final Parker parker;
         protected final Logger logger;
         protected volatile boolean shouldLogAverage = false;
         protected final ByteBuffer logBuffer;
         private RandomAccessFile ras;
         private final String displayName;
 
-        protected CoalescingStrategy(Parker parker, Logger logger, String displayName)
+        protected AbstractCoalescingStrategy(Logger logger, String displayName)
         {
-            this.parker = parker;
             this.logger = logger;
             this.displayName = displayName;
-            if (DEBUG_COALESCING)
-            {
-                NamedThreadFactory.createThread(() ->
-                {
-                    while (true)
-                    {
-                        try
-                        {
-                            Thread.sleep(5000);
-                        }
-                        catch (InterruptedException e)
-                        {
-                            throw new AssertionError();
-                        }
-                        shouldLogAverage = true;
-                    }
-                }, displayName + " debug thread").start();
-            }
+
             RandomAccessFile rasTemp = null;
             ByteBuffer logBufferTemp = null;
             if (DEBUG_COALESCING)
             {
+                ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(() -> shouldLogAverage = true, 5, 5, TimeUnit.SECONDS);
                 try
                 {
                     File outFile = File.createTempFile("coalescing_" + this.displayName + "_", ".log", new File(DEBUG_COALESCING_PATH));
@@ -214,44 +196,10 @@ public class CoalescingStrategies
                 }
             }
         }
-
-        /**
-         * Drain from the input blocking queue to the output list up to maxItems elements.
-         *
-         * The coalescing strategy may choose to park the current thread if it thinks it will
-         * be able to produce an output list with more elements.
-         *
-         * @param input Blocking queue to retrieve elements from
-         * @param out Output list to place retrieved elements in. Must be empty.
-         * @param maxItems Maximum number of elements to place in the output list
-         */
-        public <C extends Coalescable> void coalesce(BlockingQueue<C> input, List<C> out, int maxItems) throws InterruptedException
-        {
-            Preconditions.checkArgument(out.isEmpty(), "out list should be empty");
-            coalesceInternal(input, out, maxItems);
-        }
-
-        protected abstract <C extends Coalescable> void coalesceInternal(BlockingQueue<C> input, List<C> out, int maxItems) throws InterruptedException;
-
     }
 
     @VisibleForTesting
-    interface Parker
-    {
-        void park(long nanos);
-    }
-
-    private static final Parker PARKER = new Parker()
-    {
-        @Override
-        public void park(long nanos)
-        {
-            parkLoop(nanos);
-        }
-    };
-
-    @VisibleForTesting
-    static class TimeHorizonMovingAverageCoalescingStrategy extends CoalescingStrategy
+    static class TimeHorizonMovingAverageCoalescingStrategy extends AbstractCoalescingStrategy
     {
         // for now we'll just use 64ms per bucket; this can be made configurable, but results in ~1s for 16 samples
         private static final int INDEX_SHIFT = 26;
@@ -261,7 +209,7 @@ public class CoalescingStrategies
         private static final long MEASURED_INTERVAL = BUCKET_INTERVAL * (BUCKET_COUNT - 1);
 
         // the minimum timestamp we will now accept updates for; only moves forwards, never backwards
-        private long epoch = CLOCK.nanoTime();
+        private long epoch;
         // the buckets, each following on from epoch; the measurements run from ix(epoch) to ix(epoch - 1)
         // ix(epoch-1) is a partial result, that is never actually part of the calculation, and most updates
         // are expected to hit this bucket
@@ -269,31 +217,12 @@ public class CoalescingStrategies
         private long sum = 0;
         private final long maxCoalesceWindow;
 
-        public TimeHorizonMovingAverageCoalescingStrategy(int maxCoalesceWindow, Parker parker, Logger logger, String displayName)
+        public TimeHorizonMovingAverageCoalescingStrategy(int maxCoalesceWindow, Logger logger, String displayName, long initialEpoch)
         {
-            super(parker, logger, displayName);
+            super(logger, displayName);
             this.maxCoalesceWindow = TimeUnit.MICROSECONDS.toNanos(maxCoalesceWindow);
             sum = 0;
-        }
-
-        private void logSample(long nanos)
-        {
-            debugTimestamp(nanos);
-            long epoch = this.epoch;
-            long delta = nanos - epoch;
-            if (delta < 0)
-                // have to simply ignore, but would be a bit crazy to get such reordering
-                return;
-
-            if (delta > INTERVAL)
-                epoch = rollepoch(delta, epoch, nanos);
-
-            int ix = ix(nanos);
-            samples[ix]++;
-
-            // if we've updated an old bucket, we need to update the sum to match
-            if (ix != ix(epoch - 1))
-                sum++;
+            epoch = initialEpoch;
         }
 
         private long averageGap()
@@ -304,7 +233,7 @@ public class CoalescingStrategies
         }
 
         // this sample extends past the end of the range we cover, so rollover
-        private long rollepoch(long delta, long epoch, long nanos)
+        private long rollEpoch(long delta, long epoch, long nanos)
         {
             if (delta > 2 * INTERVAL)
             {
@@ -341,30 +270,32 @@ public class CoalescingStrategies
             return (int) ((nanos >>> INDEX_SHIFT) & 15);
         }
 
-        @Override
-        protected <C extends Coalescable> void coalesceInternal(BlockingQueue<C> input, List<C> out,  int maxItems) throws InterruptedException
+        public void newArrival(Coalescable message)
         {
-            if (input.drainTo(out, maxItems) == 0)
-            {
-                out.add(input.take());
-                input.drainTo(out, maxItems - out.size());
-            }
+            final long timestamp = message.timestampNanos();
+            debugTimestamp(timestamp);
+            long epoch = this.epoch;
+            long delta = timestamp - epoch;
+            if (delta < 0)
+                // have to simply ignore, but would be a bit unlucky to get such reordering
+                return;
 
-            for (Coalescable qm : out)
-                logSample(qm.timestampNanos());
+            if (delta > INTERVAL)
+                epoch = rollEpoch(delta, epoch, timestamp);
+
+            int ix = ix(timestamp);
+            samples[ix]++;
 
+            // if we've updated an old bucket, we need to update the sum to match
+            if (ix != ix(epoch - 1))
+                sum++;
+        }
+
+        public long currentCoalescingTimeNanos()
+        {
             long averageGap = averageGap();
             debugGap(averageGap);
-
-            int count = out.size();
-            if (maybeSleep(count, averageGap, maxCoalesceWindow, parker))
-            {
-                input.drainTo(out, maxItems - out.size());
-                int prevCount = count;
-                count = out.size();
-                for (int  i = prevCount; i < count; i++)
-                    logSample(out.get(i).timestampNanos());
-            }
+            return determineCoalescingTime(averageGap, maxCoalesceWindow);
         }
 
         @Override
@@ -374,25 +305,27 @@ public class CoalescingStrategies
         }
     }
 
-    /*
+    /**
      * Start coalescing by sleeping if the moving average is < the requested window.
      * The actual time spent waiting to coalesce will be the min( window, moving average * 2)
      * The actual amount of time spent waiting can be greater then the window. For instance
      * observed time spent coalescing was 400 microseconds with the window set to 200 in one benchmark.
      */
     @VisibleForTesting
-    static class MovingAverageCoalescingStrategy extends CoalescingStrategy
+    static class MovingAverageCoalescingStrategy extends AbstractCoalescingStrategy
     {
-        private final int samples[] = new int[16];
+        static final int SAMPLE_SIZE = 16;
+        private final int samples[] = new int[SAMPLE_SIZE];
+        private final long maxCoalesceWindow;
+
         private long lastSample = 0;
         private int index = 0;
         private long sum = 0;
+        private long currentGap;
 
-        private final long maxCoalesceWindow;
-
-        public MovingAverageCoalescingStrategy(int maxCoalesceWindow, Parker parker, Logger logger, String displayName)
+        public MovingAverageCoalescingStrategy(int maxCoalesceWindow, Logger logger, String displayName)
         {
-            super(parker, logger, displayName);
+            super(logger, displayName);
             this.maxCoalesceWindow = TimeUnit.MICROSECONDS.toNanos(maxCoalesceWindow);
             for (int ii = 0; ii < samples.length; ii++)
                 samples[ii] = Integer.MAX_VALUE;
@@ -406,42 +339,29 @@ public class CoalescingStrategies
             samples[index] = value;
             index++;
             index = index & ((1 << 4) - 1);
-            return sum / 16;
+            return sum / SAMPLE_SIZE;
         }
 
-        private long notifyOfSample(long sample)
+        public void newArrival(Coalescable message)
         {
-            debugTimestamp(sample);
-            if (sample > lastSample)
+            final long timestamp = message.timestampNanos();
+            debugTimestamp(timestamp);
+            if (timestamp > lastSample)
             {
-                final int delta = (int)(Math.min(Integer.MAX_VALUE, sample - lastSample));
-                lastSample = sample;
-                return logSample(delta);
+                final int delta = (int)(Math.min(Integer.MAX_VALUE, timestamp - lastSample));
+                lastSample = timestamp;
+                currentGap = logSample(delta);
             }
             else
             {
-                return logSample(1);
+                currentGap = logSample(1);
             }
         }
 
-        @Override
-        protected <C extends Coalescable> void coalesceInternal(BlockingQueue<C> input, List<C> out,  int maxItems) throws InterruptedException
+        public long currentCoalescingTimeNanos()
         {
-            if (input.drainTo(out, maxItems) == 0)
-            {
-                out.add(input.take());
-                input.drainTo(out, maxItems - out.size());
-            }
-
-            long average = notifyOfSample(out.get(0).timestampNanos());
-            debugGap(average);
-
-            if (maybeSleep(out.size(), average, maxCoalesceWindow, parker)) {
-                input.drainTo(out, maxItems - out.size());
-            }
-
-            for (int ii = 1; ii < out.size(); ii++)
-                notifyOfSample(out.get(ii).timestampNanos());
+            debugGap(currentGap);
+            return determineCoalescingTime(currentGap, maxCoalesceWindow);
         }
 
         @Override
@@ -451,35 +371,28 @@ public class CoalescingStrategies
         }
     }
 
-    /*
+    /**
      * A fixed strategy as a backup in case MovingAverage or TimeHorizongMovingAverage fails in some scenario
      */
     @VisibleForTesting
-    static class FixedCoalescingStrategy extends CoalescingStrategy
+    static class FixedCoalescingStrategy extends AbstractCoalescingStrategy
     {
         private final long coalesceWindow;
 
-        public FixedCoalescingStrategy(int coalesceWindowMicros, Parker parker, Logger logger, String displayName)
+        public FixedCoalescingStrategy(int coalesceWindowMicros, Logger logger, String displayName)
         {
-            super(parker, logger, displayName);
+            super(logger, displayName);
             coalesceWindow = TimeUnit.MICROSECONDS.toNanos(coalesceWindowMicros);
         }
 
-        @Override
-        protected <C extends Coalescable> void coalesceInternal(BlockingQueue<C> input, List<C> out,  int maxItems) throws InterruptedException
+        public void newArrival(Coalescable message)
         {
-            int enough = DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages();
+            debugTimestamp(message.timestampNanos());
+        }
 
-            if (input.drainTo(out, maxItems) == 0)
-            {
-                out.add(input.take());
-                input.drainTo(out, maxItems - out.size());
-                if (out.size() < enough) {
-                    parker.park(coalesceWindow);
-                    input.drainTo(out, maxItems - out.size());
-                }
-            }
-            debugTimestamps(out);
+        public long currentCoalescingTimeNanos()
+        {
+            return coalesceWindow;
         }
 
         @Override
@@ -489,84 +402,43 @@ public class CoalescingStrategies
         }
     }
 
-    /*
-     * A coalesscing strategy that just returns all currently available elements
-     */
-    @VisibleForTesting
-    static class DisabledCoalescingStrategy extends CoalescingStrategy
+    public static Optional<CoalescingStrategy> newCoalescingStrategy(String strategy, int coalesceWindow, Logger logger, String displayName)
     {
+        String strategyCleaned = strategy.trim().toUpperCase(Locale.ENGLISH);
 
-        public DisabledCoalescingStrategy(int coalesceWindowMicros, Parker parker, Logger logger, String displayName)
-        {
-            super(parker, logger, displayName);
-        }
-
-        @Override
-        protected <C extends Coalescable> void coalesceInternal(BlockingQueue<C> input, List<C> out,  int maxItems) throws InterruptedException
+        try
         {
-            if (input.drainTo(out, maxItems) == 0)
+            switch (Enum.valueOf(Strategy.class, strategyCleaned))
             {
-                out.add(input.take());
-                input.drainTo(out, maxItems - 1);
+                case MOVINGAVERAGE:
+                    return Optional.of(new MovingAverageCoalescingStrategy(coalesceWindow, logger, displayName));
+                case FIXED:
+                    return Optional.of(new FixedCoalescingStrategy(coalesceWindow, logger, displayName));
+                case TIMEHORIZON:
+                    long initialEpoch = System.nanoTime();
+                    return Optional.of(new TimeHorizonMovingAverageCoalescingStrategy(coalesceWindow, logger, displayName, initialEpoch));
+                case DISABLED:
+                    return Optional.empty();
+                default:
+                    throw new IllegalArgumentException("supported coalese strategy");
             }
-            debugTimestamps(out);
         }
-
-        @Override
-        public String toString()
+        catch (IllegalArgumentException iae)
         {
-            return "Disabled";
-        }
-    }
-
-    @VisibleForTesting
-    static CoalescingStrategy newCoalescingStrategy(String strategy,
-                                                    int coalesceWindow,
-                                                    Parker parker,
-                                                    Logger logger,
-                                                    String displayName)
-    {
-        String classname = null;
-        String strategyCleaned = strategy.trim().toUpperCase(Locale.ENGLISH);
-        switch(strategyCleaned)
-        {
-        case "MOVINGAVERAGE":
-            classname = MovingAverageCoalescingStrategy.class.getName();
-            break;
-        case "FIXED":
-            classname = FixedCoalescingStrategy.class.getName();
-            break;
-        case "TIMEHORIZON":
-            classname = TimeHorizonMovingAverageCoalescingStrategy.class.getName();
-            break;
-        case "DISABLED":
-            classname = DisabledCoalescingStrategy.class.getName();
-            break;
-        default:
-            classname = strategy;
-        }
+            try
+            {
+                Class<?> clazz = Class.forName(strategy);
 
-        try
-        {
-            Class<?> clazz = Class.forName(classname);
+                if (!CoalescingStrategy.class.isAssignableFrom(clazz))
+                    throw new RuntimeException(strategy + " is not an instance of CoalescingStrategy");
 
-            if (!CoalescingStrategy.class.isAssignableFrom(clazz))
+                Constructor<?> constructor = clazz.getConstructor(int.class, Logger.class, String.class);
+                return Optional.of((CoalescingStrategy)constructor.newInstance(coalesceWindow, logger, displayName));
+            }
+            catch (Exception e)
             {
-                throw new RuntimeException(classname + " is not an instance of CoalescingStrategy");
+                throw new RuntimeException(e);
             }
-
-            Constructor<?> constructor = clazz.getConstructor(int.class, Parker.class, Logger.class, String.class);
-
-            return (CoalescingStrategy)constructor.newInstance(coalesceWindow, parker, logger, displayName);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
         }
     }
-
-    public static CoalescingStrategy newCoalescingStrategy(String strategy, int coalesceWindow, Logger logger, String displayName)
-    {
-        return newCoalescingStrategy(strategy, coalesceWindow, PARKER, logger, displayName);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/utils/FBUtilities.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 3fa64b3..58c3371 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -147,6 +147,13 @@ public class FBUtilities
         return broadcastInetAddress;
     }
 
+    /**
+     * <b>THIS IS FOR TESTING ONLY!!</b>
+     */
+    public static void setBroadcastInetAddress(InetAddress addr)
+    {
+        broadcastInetAddress = addr;
+    }
 
     public static InetAddress getBroadcastRpcAddress()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/src/java/org/apache/cassandra/utils/NativeLibrary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeLibrary.java b/src/java/org/apache/cassandra/utils/NativeLibrary.java
index 735d51a..7d54791 100644
--- a/src/java/org/apache/cassandra/utils/NativeLibrary.java
+++ b/src/java/org/apache/cassandra/utils/NativeLibrary.java
@@ -49,7 +49,7 @@ public final class NativeLibrary
         OTHER;
     }
 
-    private static final OSType osType;
+    public static final OSType osType;
 
     private static final int MCL_CURRENT;
     private static final int MCL_FUTURE;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/conf/cassandra_ssl_test.keystore
----------------------------------------------------------------------
diff --git a/test/conf/cassandra_ssl_test.keystore b/test/conf/cassandra_ssl_test.keystore
new file mode 100644
index 0000000..8b2b218
Binary files /dev/null and b/test/conf/cassandra_ssl_test.keystore differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/conf/cassandra_ssl_test.truststore
----------------------------------------------------------------------
diff --git a/test/conf/cassandra_ssl_test.truststore b/test/conf/cassandra_ssl_test.truststore
new file mode 100644
index 0000000..49cf332
Binary files /dev/null and b/test/conf/cassandra_ssl_test.truststore differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/db/ReadCommandTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadCommandTest.java b/test/unit/org/apache/cassandra/db/ReadCommandTest.java
index 249d780..056089e 100644
--- a/test/unit/org/apache/cassandra/db/ReadCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadCommandTest.java
@@ -18,10 +18,13 @@
 
 package org.apache.cassandra.db;
 
+import java.io.IOException;
+import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -46,6 +49,7 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterators;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.TableMetadata;
@@ -306,4 +310,33 @@ public class ReadCommandTest
             assertEquals(expectedRows.length, i);
         }
     }
+
+    public void serializerTest() throws IOException
+    {
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF2);
+
+        new RowUpdateBuilder(cfs.metadata.get(), 0, ByteBufferUtil.bytes("key"))
+        .clustering("dd")
+        .add("a", ByteBufferUtil.bytes("abcd"))
+        .build()
+        .apply();
+
+        ReadCommand readCommand = Util.cmd(cfs, Util.dk("key")).includeRow("dd").build();
+        int messagingVersion = MessagingService.current_version;
+        long size = ReadCommand.serializer.serializedSize(readCommand, messagingVersion);
+
+        FakeOutputStream out = new FakeOutputStream();
+        ReadCommand.serializer.serialize(readCommand, new WrappedDataOutputStreamPlus(out), messagingVersion);
+        Assert.assertEquals(size, out.count);
+    }
+
+    static class FakeOutputStream extends OutputStream
+    {
+        long count;
+
+        public void write(int b) throws IOException
+        {
+            count++;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
index 5e99523..09973a8 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -21,12 +21,10 @@ package org.apache.cassandra.locator;
 
 import java.io.IOException;
 import java.net.InetAddress;
-import java.net.UnknownHostException;
 import java.util.EnumMap;
 import java.util.Map;
 
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -37,8 +35,6 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.OutboundTcpConnectionPool;
 import org.apache.cassandra.service.StorageService;
 
 import static org.junit.Assert.assertEquals;
@@ -102,22 +98,6 @@ public class EC2SnitchTest
         assertEquals("2d", snitch.getRack(local));
     }
 
-    @Test
-    public void testEc2MRSnitch() throws UnknownHostException
-    {
-        InetAddress me = InetAddress.getByName("127.0.0.2");
-        InetAddress com_ip = InetAddress.getByName("127.0.0.3");
-
-        OutboundTcpConnectionPool pool = MessagingService.instance().getConnectionPool(me);
-        Assert.assertEquals(me, pool.endPoint());
-        pool.reset(com_ip);
-        Assert.assertEquals(com_ip, pool.endPoint());
-
-        MessagingService.instance().destroyConnectionPool(me);
-        pool = MessagingService.instance().getConnectionPool(me);
-        Assert.assertEquals(com_ip, pool.endPoint());
-    }
-
     @AfterClass
     public static void tearDown()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
index 7cb3cfd..a082d56 100644
--- a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
+++ b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
@@ -20,10 +20,9 @@
  */
 package org.apache.cassandra.net;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.Arrays;
 import java.util.Collections;
@@ -31,23 +30,31 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.*;
 import java.util.regex.Matcher;
 
 import com.google.common.collect.Iterables;
+
 import com.codahale.metrics.Timer;
 
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
 import org.apache.cassandra.auth.IInternodeAuthenticator;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.monitoring.ApproximateTime;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
+import org.apache.cassandra.net.MessagingService.ServerChannel;
+import org.apache.cassandra.net.async.NettyFactory;
+import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
+import org.apache.cassandra.net.async.OutboundConnectionParams;
+import org.apache.cassandra.net.async.OutboundMessagingPool;
+import org.apache.cassandra.utils.FBUtilities;
 import org.caffinitas.ohc.histo.EstimatedHistogram;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -93,6 +100,12 @@ public class MessagingServiceTest
         messagingService.destroyConnectionPool(InetAddress.getByName("127.0.0.3"));
     }
 
+    @After
+    public void replaceAuthenticator()
+    {
+        DatabaseDescriptor.setInternodeAuthenticator(originalAuthenticator);
+    }
+
     @Test
     public void testDroppedMessages()
     {
@@ -197,7 +210,7 @@ public class MessagingServiceTest
     @Test
     public void testUpdatesBackPressureOnSendWhenEnabledAndWithSupportedCallback() throws UnknownHostException
     {
-        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getConnectionPool(InetAddress.getByName("127.0.0.2")).getBackPressureState();
+        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddress.getByName("127.0.0.2"));
         IAsyncCallback bpCallback = new BackPressureCallback();
         IAsyncCallback noCallback = new NoBackPressureCallback();
         MessageOut<?> ignored = null;
@@ -218,7 +231,7 @@ public class MessagingServiceTest
     @Test
     public void testUpdatesBackPressureOnReceiveWhenEnabledAndWithSupportedCallback() throws UnknownHostException
     {
-        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getConnectionPool(InetAddress.getByName("127.0.0.2")).getBackPressureState();
+        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddress.getByName("127.0.0.2"));
         IAsyncCallback bpCallback = new BackPressureCallback();
         IAsyncCallback noCallback = new NoBackPressureCallback();
         boolean timeout = false;
@@ -242,7 +255,7 @@ public class MessagingServiceTest
     @Test
     public void testUpdatesBackPressureOnTimeoutWhenEnabledAndWithSupportedCallback() throws UnknownHostException
     {
-        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getConnectionPool(InetAddress.getByName("127.0.0.2")).getBackPressureState();
+        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddress.getByName("127.0.0.2"));
         IAsyncCallback bpCallback = new BackPressureCallback();
         IAsyncCallback noCallback = new NoBackPressureCallback();
         boolean timeout = true;
@@ -285,13 +298,7 @@ public class MessagingServiceTest
 
     private static void addDCLatency(long sentAt, long nowTime) throws IOException
     {
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        try (DataOutputStreamPlus out = new WrappedDataOutputStreamPlus(baos))
-        {
-            out.writeInt((int) sentAt);
-        }
-        DataInputStreamPlus in = new DataInputStreamPlus(new ByteArrayInputStream(baos.toByteArray()));
-        MessageIn.readConstructionTime(InetAddress.getLocalHost(), in, nowTime);
+        MessageIn.deriveConstructionTime(InetAddress.getLocalHost(), (int)sentAt, nowTime);
     }
 
     public static class MockBackPressureStrategy implements BackPressureStrategy<MockBackPressureStrategy.MockBackPressureState>
@@ -414,32 +421,83 @@ public class MessagingServiceTest
         InetAddress address = InetAddress.getByName("127.0.0.250");
 
         //Should return null
-        assertNull(ms.getConnectionPool(address));
-        assertNull(ms.getConnection(address, new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK)));
+        MessageOut messageOut = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK);
+        assertFalse(ms.isConnected(address, messageOut));
 
         //Should tolerate null
         ms.convict(address);
-        ms.sendOneWay(new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK), address);
+        ms.sendOneWay(messageOut, address);
     }
 
     @Test
-    public void testOutboundTcpConnectionCleansUp() throws Exception
+    public void testOutboundMessagingConnectionCleansUp() throws Exception
     {
         MessagingService ms = MessagingService.instance();
-        DatabaseDescriptor.setInternodeAuthenticator(ALLOW_NOTHING_AUTHENTICATOR);
-        InetAddress address = InetAddress.getByName("127.0.0.250");
-        OutboundTcpConnectionPool pool = new OutboundTcpConnectionPool(address, new MockBackPressureStrategy(null).newState(address));
-        ms.connectionManagers.put(address, pool);
-        pool.smallMessages.start();
-        pool.smallMessages.enqueue(new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK), 0);
-        pool.smallMessages.join();
-        assertFalse(ms.connectionManagers.containsKey(address));
+        InetSocketAddress local = new InetSocketAddress("127.0.0.1", 9876);
+        InetSocketAddress remote = new InetSocketAddress("127.0.0.2", 9876);
+
+        OutboundMessagingPool pool = new OutboundMessagingPool(remote, local, null, new MockBackPressureStrategy(null).newState(remote.getAddress()), ALLOW_NOTHING_AUTHENTICATOR);
+        ms.channelManagers.put(remote.getAddress(), pool);
+        pool.sendMessage(new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK), 0);
+        assertFalse(ms.channelManagers.containsKey(remote.getAddress()));
     }
 
-    @After
-    public void replaceAuthenticator()
+    @Test
+    public void reconnectWithNewIp() throws UnknownHostException
     {
-        DatabaseDescriptor.setInternodeAuthenticator(originalAuthenticator);
+        InetAddress publicIp = InetAddress.getByName("127.0.0.2");
+        InetAddress privateIp = InetAddress.getByName("127.0.0.3");
+
+        // reset the preferred IP value, for good test hygene
+        SystemKeyspace.updatePreferredIP(publicIp, publicIp);
+
+        // create pool/conn with public addr
+        Assert.assertEquals(publicIp, messagingService.getCurrentEndpoint(publicIp));
+        messagingService.reconnectWithNewIp(publicIp, privateIp);
+        Assert.assertEquals(privateIp, messagingService.getCurrentEndpoint(publicIp));
+
+        messagingService.destroyConnectionPool(publicIp);
+
+        // recreate the pool/conn, and make sure the preferred ip addr is used
+        Assert.assertEquals(privateIp, messagingService.getCurrentEndpoint(publicIp));
+    }
+
+    @Test
+    public void testCloseInboundConnections() throws UnknownHostException, InterruptedException
+    {
+        messagingService.listen();
+        Assert.assertTrue(messagingService.isListening());
+        Assert.assertTrue(messagingService.serverChannels.size() > 0);
+        for (ServerChannel serverChannel : messagingService.serverChannels)
+            Assert.assertEquals(0, serverChannel.size());
+
+        // now, create a connection and make sure it's in a channel group
+        InetSocketAddress server = new InetSocketAddress(FBUtilities.getBroadcastAddress(), DatabaseDescriptor.getStoragePort());
+        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.small(new InetSocketAddress(InetAddress.getByName("127.0.0.2"), 0), server);
+
+        CountDownLatch latch = new CountDownLatch(1);
+        OutboundConnectionParams params = OutboundConnectionParams.builder()
+                                                                  .mode(NettyFactory.Mode.MESSAGING)
+                                                                  .sendBufferSize(1 << 10)
+                                                                  .connectionId(id)
+                                                                  .callback(handshakeResult -> latch.countDown())
+                                                                  .protocolVersion(MessagingService.current_version)
+                                                                  .build();
+        Bootstrap bootstrap = NettyFactory.instance.createOutboundBootstrap(params);
+        Channel channel = bootstrap.connect().awaitUninterruptibly().channel();
+        Assert.assertNotNull(channel);
+        latch.await(1, TimeUnit.SECONDS); // allow the netty pipeline/c* handshake to get set up
+
+        int connectCount = 0;
+        for (ServerChannel serverChannel : messagingService.serverChannels)
+            connectCount += serverChannel.size();
+        Assert.assertTrue(connectCount > 0);
+
+        // last, shutdown the MS and make sure connections are removed
+        messagingService.shutdown();
+        for (ServerChannel serverChannel : messagingService.serverChannels)
+            Assert.assertEquals(0, serverChannel.size());
     }
 
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/OutboundTcpConnectionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/OutboundTcpConnectionTest.java b/test/unit/org/apache/cassandra/net/OutboundTcpConnectionTest.java
deleted file mode 100644
index e3b6817..0000000
--- a/test/unit/org/apache/cassandra/net/OutboundTcpConnectionTest.java
+++ /dev/null
@@ -1,175 +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.net;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.net.MessagingService.Verb;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * The tests check whether Queue expiration in the OutboundTcpConnection behaves properly for droppable and
- * non-droppable messages.
- */
-public class OutboundTcpConnectionTest
-{
-    AtomicInteger messageId = new AtomicInteger(0);
-
-    final static Verb VERB_DROPPABLE = Verb.MUTATION; // Droppable, 2s timeout
-    final static Verb VERB_NONDROPPABLE = Verb.GOSSIP_DIGEST_ACK; // Not droppable
-    
-    final static long NANOS_FOR_TIMEOUT;
-
-    static
-    {
-        DatabaseDescriptor.daemonInitialization();
-        NANOS_FOR_TIMEOUT = TimeUnit.MILLISECONDS.toNanos(VERB_DROPPABLE.getTimeout()*2);
-    }
-    
-    /**
-     * Verifies our assumptions whether a Verb can be dropped or not. The tests make use of droppabilty, and
-     * may produce wrong test results if their droppabilty is changed. 
-     */
-    @BeforeClass
-    public static void assertDroppability()
-    {
-        if (!MessagingService.DROPPABLE_VERBS.contains(VERB_DROPPABLE))
-            throw new AssertionError("Expected " + VERB_DROPPABLE + " to be droppable");
-        if (MessagingService.DROPPABLE_VERBS.contains(VERB_NONDROPPABLE))
-            throw new AssertionError("Expected " + VERB_NONDROPPABLE + " not to be droppable");
-    }
-
-    /**
-     * Tests that non-droppable messages are never expired
-     */
-    @Test
-    public void testNondroppable() throws UnknownHostException
-    {
-        OutboundTcpConnection otc = getOutboundTcpConnectionForLocalhost();
-        long nanoTimeBeforeEnqueue = System.nanoTime();
-
-        assertFalse("Fresh OutboundTcpConnection contains expired messages",
-                otc.backlogContainsExpiredMessages(nanoTimeBeforeEnqueue));
-
-        fillToPurgeSize(otc, VERB_NONDROPPABLE);
-        fillToPurgeSize(otc, VERB_NONDROPPABLE);
-        otc.expireMessages(expirationTimeNanos());
-
-        assertFalse("OutboundTcpConnection with non-droppable verbs should not expire",
-                otc.backlogContainsExpiredMessages(expirationTimeNanos()));
-    }
-
-    /**
-     * Tests that droppable messages will be dropped after they expire, but not before.
-     * 
-     * @throws UnknownHostException
-     */
-    @Test
-    public void testDroppable() throws UnknownHostException
-    {
-        OutboundTcpConnection otc = getOutboundTcpConnectionForLocalhost();
-        long nanoTimeBeforeEnqueue = System.nanoTime();
-
-        initialFill(otc, VERB_DROPPABLE);
-        assertFalse("OutboundTcpConnection with droppable verbs should not expire immediately",
-                otc.backlogContainsExpiredMessages(nanoTimeBeforeEnqueue));
-
-        otc.expireMessages(nanoTimeBeforeEnqueue);
-        assertFalse("OutboundTcpConnection with droppable verbs should not expire with enqueue-time expiration",
-                otc.backlogContainsExpiredMessages(nanoTimeBeforeEnqueue));
-
-        // Lets presume, expiration time have passed => At that time there shall be expired messages in the Queue
-        long nanoTimeWhenExpired = expirationTimeNanos();
-        assertTrue("OutboundTcpConnection with droppable verbs should have expired",
-                otc.backlogContainsExpiredMessages(nanoTimeWhenExpired));
-
-        // Using the same timestamp, lets expire them and check whether they have gone
-        otc.expireMessages(nanoTimeWhenExpired);
-        assertFalse("OutboundTcpConnection should not have expired entries",
-                otc.backlogContainsExpiredMessages(nanoTimeWhenExpired));
-
-        // Actually the previous test can be done in a harder way: As expireMessages() has run, we cannot have
-        // ANY expired values, thus lets test also against nanoTimeBeforeEnqueue
-        assertFalse("OutboundTcpConnection should not have any expired entries",
-                otc.backlogContainsExpiredMessages(nanoTimeBeforeEnqueue));
-
-    }
-
-    /**
-     * Fills the given OutboundTcpConnection with (1 + BACKLOG_PURGE_SIZE), elements. The first
-     * BACKLOG_PURGE_SIZE elements are non-droppable, the last one is a message with the given Verb and can be
-     * droppable or non-droppable.
-     */
-    private void initialFill(OutboundTcpConnection otc, Verb verb)
-    {
-        assertFalse("Fresh OutboundTcpConnection contains expired messages",
-                otc.backlogContainsExpiredMessages(System.nanoTime()));
-
-        fillToPurgeSize(otc, VERB_NONDROPPABLE);
-        MessageOut<?> messageDroppable10s = new MessageOut<>(verb);
-        otc.enqueue(messageDroppable10s, nextMessageId());
-        otc.expireMessages(System.nanoTime());
-    }
-
-    /**
-     * Returns a nano timestamp in the far future, when expiration should have been performed for VERB_DROPPABLE.
-     * The offset is chosen as 2 times of the expiration time of VERB_DROPPABLE.
-     * 
-     * @return The future nano timestamp
-     */
-    private long expirationTimeNanos()
-    {
-        return System.nanoTime() + NANOS_FOR_TIMEOUT;
-    }
-
-    private int nextMessageId()
-    {
-        return messageId.incrementAndGet();
-    }
-
-    /**
-     * Adds BACKLOG_PURGE_SIZE messages to the queue. Hint: At BACKLOG_PURGE_SIZE expiration starts to work.
-     * 
-     * @param otc
-     *            The OutboundTcpConnection
-     * @param verb
-     *            The verb that defines the message type
-     */
-    private void fillToPurgeSize(OutboundTcpConnection otc, Verb verb)
-    {
-        for (int i = 0; i < OutboundTcpConnection.BACKLOG_PURGE_SIZE; i++)
-        {
-            otc.enqueue(new MessageOut<>(verb), nextMessageId());
-        }
-    }
-
-    private OutboundTcpConnection getOutboundTcpConnectionForLocalhost() throws UnknownHostException
-    {
-        InetAddress lo = InetAddress.getByName("127.0.0.1");
-        OutboundTcpConnectionPool otcPool = new OutboundTcpConnectionPool(lo, null);
-        OutboundTcpConnection otc = new OutboundTcpConnection(otcPool, "lo-OutboundTcpConnectionTest");
-        return otc;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/ByteBufDataOutputPlusTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/ByteBufDataOutputPlusTest.java b/test/unit/org/apache/cassandra/net/async/ByteBufDataOutputPlusTest.java
new file mode 100644
index 0000000..959c37a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/ByteBufDataOutputPlusTest.java
@@ -0,0 +1,178 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.Memory;
+import org.apache.cassandra.io.util.SafeMemory;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.KeyspaceParams;
+
+public class ByteBufDataOutputPlusTest
+{
+    private static final String KEYSPACE1 = "NettyPipilineTest";
+    private static final String STANDARD1 = "Standard1";
+    private static final int columnCount = 128;
+
+    private ByteBuf buf;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, STANDARD1, columnCount, AsciiType.instance, BytesType.instance));
+        CompactionManager.instance.disableAutoCompaction();
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (buf != null)
+            buf.release();
+    }
+
+    @Test
+    public void compareBufferSizes() throws IOException
+    {
+        final int currentFrameSize = getMessage().message.serializedSize(MessagingService.current_version);
+
+        ByteBuffer buffer = ByteBuffer.allocateDirect(currentFrameSize); //bufferedOut.nioBuffer(0, bufferedOut.writableBytes());
+        getMessage().message.serialize(new DataOutputBuffer(buffer), MessagingService.current_version);
+        Assert.assertFalse(buffer.hasRemaining());
+        Assert.assertEquals(buffer.capacity(), buffer.position());
+
+        ByteBuf bbosOut = PooledByteBufAllocator.DEFAULT.ioBuffer(currentFrameSize, currentFrameSize);
+        try
+        {
+            getMessage().message.serialize(new ByteBufDataOutputPlus(bbosOut), MessagingService.current_version);
+
+            Assert.assertFalse(bbosOut.isWritable());
+            Assert.assertEquals(bbosOut.capacity(), bbosOut.writerIndex());
+
+            Assert.assertEquals(buffer.position(), bbosOut.writerIndex());
+            for (int i = 0; i < currentFrameSize; i++)
+            {
+                Assert.assertEquals(buffer.get(i), bbosOut.getByte(i));
+            }
+        }
+        finally
+        {
+            bbosOut.release();
+        }
+    }
+
+    private QueuedMessage getMessage()
+    {
+        ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+        ByteBuffer buf = ByteBuffer.allocate(1 << 10);
+        RowUpdateBuilder rowUpdateBuilder = new RowUpdateBuilder(cfs1.metadata.get(), 0, "k")
+                                            .clustering("bytes");
+        for (int i = 0; i < columnCount; i++)
+            rowUpdateBuilder.add("val" + i, buf);
+
+        Mutation mutation = rowUpdateBuilder.build();
+        return new QueuedMessage(mutation.createMessage(), 42);
+    }
+
+    @Test
+    public void compareDOS() throws IOException
+    {
+        buf = PooledByteBufAllocator.DEFAULT.ioBuffer(1024, 1024);
+        ByteBuffer buffer = ByteBuffer.allocateDirect(1024);
+
+        ByteBufDataOutputPlus byteBufDataOutputPlus = new ByteBufDataOutputPlus(buf);
+        DataOutputBuffer dataOutputBuffer = new DataOutputBuffer(buffer);
+
+        write(byteBufDataOutputPlus);
+        write(dataOutputBuffer);
+
+        Assert.assertEquals(buffer.position(), buf.writerIndex());
+        for (int i = 0; i < buffer.position(); i++)
+        {
+            Assert.assertEquals(buffer.get(i), buf.getByte(i));
+        }
+    }
+
+    private void write(DataOutputPlus out) throws IOException
+    {
+        ByteBuffer b = ByteBuffer.allocate(8);
+        b.putLong(29811134237462734L);
+        out.write(b);
+        b = ByteBuffer.allocateDirect(8);
+        b.putDouble(92367.4253647890626);
+        out.write(b);
+
+        out.writeInt(29319236);
+
+        byte[] array = new byte[17];
+        for (int i = 0; i < array.length; i++)
+            array[i] = (byte)i;
+        out.write(array, 0 , array.length);
+
+        out.write(42);
+        out.writeUTF("This is a great string!!");
+        out.writeByte(-100);
+        out.writeUnsignedVInt(3247634L);
+        out.writeVInt(12313695L);
+        out.writeBoolean(true);
+        out.writeShort(4371);
+        out.writeChar('j');
+        out.writeLong(472348263487234L);
+        out.writeFloat(34534.12623F);
+        out.writeDouble(0.2384253D);
+        out.writeBytes("Write my bytes");
+        out.writeChars("These are some swell chars");
+
+        Memory memory = new SafeMemory(8);
+        memory.setLong(0, -21365123651231L);
+        out.write(memory, 0, memory.size());
+        memory.close();
+    }
+
+    @Test (expected = UnsupportedOperationException.class)
+    public void applyToChannel() throws IOException
+    {
+        ByteBufDataOutputPlus out = new ByteBufDataOutputPlus(Unpooled.wrappedBuffer(new byte[0]));
+        out.applyToChannel(null);
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org

[06/11] cassandra git commit: switch internode messaging to netty

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java b/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java
new file mode 100644
index 0000000..128fe4b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java
@@ -0,0 +1,312 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.WriteBufferWaterMark;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.ChannelWriter.CoalescingChannelWriter;
+import org.apache.cassandra.utils.CoalescingStrategies;
+import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
+
+import static org.apache.cassandra.net.MessagingService.Verb.ECHO;
+
+/**
+ * with the write_Coalescing_* methods, if there's data in the channel.unsafe().outboundBuffer()
+ * it means that there's something in the channel that hasn't yet been flushed to the transport (socket).
+ * once a flush occurs, there will be an entry in EmbeddedChannel's outboundQueue. those two facts are leveraged in these tests.
+ */
+public class ChannelWriterTest
+{
+    private static final int COALESCE_WINDOW_MS = 10;
+
+    private EmbeddedChannel channel;
+    private ChannelWriter channelWriter;
+    private NonSendingOutboundMessagingConnection omc;
+    private Optional<CoalescingStrategy> coalescingStrategy;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Before
+    public void setup()
+    {
+        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.small(new InetSocketAddress("127.0.0.1", 0),
+                                                                             new InetSocketAddress("127.0.0.2", 0));
+        channel = new EmbeddedChannel();
+        omc = new NonSendingOutboundMessagingConnection(id, null, Optional.empty());
+        channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty());
+        channel.pipeline().addFirst(new MessageOutHandler(id, MessagingService.current_version, channelWriter, () -> null));
+        coalescingStrategy = CoalescingStrategies.newCoalescingStrategy(CoalescingStrategies.Strategy.FIXED.name(), COALESCE_WINDOW_MS, null, "test");
+    }
+
+    @Test
+    public void create_nonCoalescing()
+    {
+        Assert.assertSame(ChannelWriter.SimpleChannelWriter.class, ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()).getClass());
+    }
+
+    @Test
+    public void create_Coalescing()
+    {
+        Assert.assertSame(CoalescingChannelWriter.class, ChannelWriter.create(channel, omc::handleMessageResult, coalescingStrategy).getClass());
+    }
+
+    @Test
+    public void write_IsWritable()
+    {
+        Assert.assertTrue(channel.isWritable());
+        Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true));
+        Assert.assertTrue(channel.isWritable());
+        Assert.assertTrue(channel.releaseOutbound());
+    }
+
+    @Test
+    public void write_NotWritable()
+    {
+        channel.config().setOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(1, 2));
+
+        // send one message through, which will trigger the writability check (and turn it off)
+        Assert.assertTrue(channel.isWritable());
+        ByteBuf buf = channel.alloc().buffer(8, 8);
+        channel.unsafe().outboundBuffer().addMessage(buf, buf.capacity(), channel.newPromise());
+        Assert.assertFalse(channel.isWritable());
+        Assert.assertFalse(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true));
+        Assert.assertFalse(channel.isWritable());
+        Assert.assertFalse(channel.releaseOutbound());
+        buf.release();
+    }
+
+    @Test
+    public void write_NotWritableButWriteAnyway()
+    {
+        channel.config().setOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(1, 2));
+
+        // send one message through, which will trigger the writability check (and turn it off)
+        Assert.assertTrue(channel.isWritable());
+        ByteBuf buf = channel.alloc().buffer(8, 8);
+        channel.unsafe().outboundBuffer().addMessage(buf, buf.capacity(), channel.newPromise());
+        Assert.assertFalse(channel.isWritable());
+        Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), false));
+        Assert.assertTrue(channel.isWritable());
+        Assert.assertTrue(channel.releaseOutbound());
+    }
+
+    @Test
+    public void write_Coalescing_LostRaceForFlushTask()
+    {
+        CoalescingChannelWriter channelWriter = resetEnvForCoalescing(DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages());
+        channelWriter.scheduledFlush.set(true);
+        Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0);
+        Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true));
+        Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() > 0);
+        Assert.assertFalse(channel.releaseOutbound());
+        Assert.assertTrue(channelWriter.scheduledFlush.get());
+    }
+
+    @Test
+    public void write_Coalescing_HitMinMessageCountForImmediateCoalesce()
+    {
+        CoalescingChannelWriter channelWriter = resetEnvForCoalescing(1);
+
+        Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0);
+        Assert.assertFalse(channelWriter.scheduledFlush.get());
+        Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true));
+
+        Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0);
+        Assert.assertTrue(channel.releaseOutbound());
+        Assert.assertFalse(channelWriter.scheduledFlush.get());
+    }
+
+    @Test
+    public void write_Coalescing_ScheduleFlushTask()
+    {
+        CoalescingChannelWriter channelWriter = resetEnvForCoalescing(DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages());
+
+        Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0);
+        Assert.assertFalse(channelWriter.scheduledFlush.get());
+        Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true));
+
+        Assert.assertTrue(channelWriter.scheduledFlush.get());
+        Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() > 0);
+        Assert.assertTrue(channelWriter.scheduledFlush.get());
+
+        // this unfortunately know a little too much about how the sausage is made in CoalescingChannelWriter :-/
+        channel.runScheduledPendingTasks();
+        channel.runPendingTasks();
+        Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0);
+        Assert.assertFalse(channelWriter.scheduledFlush.get());
+        Assert.assertTrue(channel.releaseOutbound());
+    }
+
+    private CoalescingChannelWriter resetEnvForCoalescing(int minMessagesForCoalesce)
+    {
+        channel = new EmbeddedChannel();
+        CoalescingChannelWriter cw = new CoalescingChannelWriter(channel, omc::handleMessageResult, coalescingStrategy.get(), minMessagesForCoalesce);
+        channel.pipeline().addFirst(new ChannelOutboundHandlerAdapter()
+        {
+            public void flush(ChannelHandlerContext ctx) throws Exception
+            {
+                cw.onTriggeredFlush(ctx);
+            }
+        });
+        omc.setChannelWriter(cw);
+        return cw;
+    }
+
+    @Test
+    public void writeBacklog_Empty()
+    {
+        BlockingQueue<QueuedMessage> queue = new LinkedBlockingQueue<>();
+        Assert.assertEquals(0, channelWriter.writeBacklog(queue, false));
+        Assert.assertFalse(channel.releaseOutbound());
+    }
+
+    @Test
+    public void writeBacklog_ChannelNotWritable()
+    {
+        Assert.assertTrue(channel.isWritable());
+        // force the channel to be non writable
+        channel.config().setOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(1, 2));
+        ByteBuf buf = channel.alloc().buffer(8, 8);
+        channel.unsafe().outboundBuffer().addMessage(buf, buf.capacity(), channel.newPromise());
+        Assert.assertFalse(channel.isWritable());
+
+        Assert.assertEquals(0, channelWriter.writeBacklog(new LinkedBlockingQueue<>(), false));
+        Assert.assertFalse(channel.releaseOutbound());
+        Assert.assertFalse(channel.isWritable());
+        buf.release();
+    }
+
+    @Test
+    public void writeBacklog_NotEmpty()
+    {
+        BlockingQueue<QueuedMessage> queue = new LinkedBlockingQueue<>();
+        int count = 12;
+        for (int i = 0; i < count; i++)
+            queue.offer(new QueuedMessage(new MessageOut<>(ECHO), i));
+        Assert.assertEquals(count, channelWriter.writeBacklog(queue, false));
+        Assert.assertTrue(channel.releaseOutbound());
+    }
+
+    @Test
+    public void close()
+    {
+        Assert.assertFalse(channelWriter.isClosed());
+        Assert.assertTrue(channel.isOpen());
+        channelWriter.close();
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertTrue(channelWriter.isClosed());
+    }
+
+    @Test
+    public void softClose()
+    {
+        Assert.assertFalse(channelWriter.isClosed());
+        Assert.assertTrue(channel.isOpen());
+        channelWriter.softClose();
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertTrue(channelWriter.isClosed());
+    }
+
+    @Test
+    public void handleMessagePromise_FutureIsCancelled()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.cancel(false);
+        channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1), true);
+        Assert.assertTrue(channel.isActive());
+        Assert.assertEquals(1, omc.getCompletedMessages().longValue());
+        Assert.assertEquals(0, omc.getDroppedMessages().longValue());
+    }
+
+    @Test
+    public void handleMessagePromise_ExpiredException_DoNotRetryMsg()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.setFailure(new ExpiredException());
+
+        channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1), true);
+        Assert.assertTrue(channel.isActive());
+        Assert.assertEquals(1, omc.getCompletedMessages().longValue());
+        Assert.assertEquals(1, omc.getDroppedMessages().longValue());
+        Assert.assertFalse(omc.sendMessageInvoked);
+    }
+
+    @Test
+    public void handleMessagePromise_NonIOException()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.setFailure(new NullPointerException("this is a test"));
+        channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1), true);
+        Assert.assertTrue(channel.isActive());
+        Assert.assertEquals(1, omc.getCompletedMessages().longValue());
+        Assert.assertEquals(0, omc.getDroppedMessages().longValue());
+        Assert.assertFalse(omc.sendMessageInvoked);
+    }
+
+    @Test
+    public void handleMessagePromise_IOException_ChannelNotClosed_RetryMsg()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.setFailure(new IOException("this is a test"));
+        Assert.assertTrue(channel.isActive());
+        channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1, 0, true, true), true);
+
+        Assert.assertFalse(channel.isActive());
+        Assert.assertEquals(1, omc.getCompletedMessages().longValue());
+        Assert.assertEquals(0, omc.getDroppedMessages().longValue());
+        Assert.assertTrue(omc.sendMessageInvoked);
+    }
+
+    @Test
+    public void handleMessagePromise_Cancelled()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.cancel(false);
+        Assert.assertTrue(channel.isActive());
+        channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1, 0, true, true), true);
+
+        Assert.assertTrue(channel.isActive());
+        Assert.assertEquals(1, omc.getCompletedMessages().longValue());
+        Assert.assertEquals(0, omc.getDroppedMessages().longValue());
+        Assert.assertFalse(omc.sendMessageInvoked);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java b/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
new file mode 100644
index 0000000..fa6e2b5
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.Optional;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.KeyspaceParams;
+
+import static org.apache.cassandra.net.async.InboundHandshakeHandler.State.MESSAGING_HANDSHAKE_COMPLETE;
+import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.READY;
+
+public class HandshakeHandlersTest
+{
+    private static final String KEYSPACE1 = "NettyPipilineTest";
+    private static final String STANDARD1 = "Standard1";
+
+    private static final InetSocketAddress LOCAL_ADDR = new InetSocketAddress("127.0.0.1", 9999);
+    private static final InetSocketAddress REMOTE_ADDR = new InetSocketAddress("127.0.0.2", 9999);
+    private static final int MESSAGING_VERSION = MessagingService.current_version;
+    private static final OutboundConnectionIdentifier connectionId = OutboundConnectionIdentifier.small(LOCAL_ADDR, REMOTE_ADDR);
+
+    @BeforeClass
+    public static void beforeClass() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, STANDARD1, 0, AsciiType.instance, BytesType.instance));
+        CompactionManager.instance.disableAutoCompaction();
+    }
+
+    @Test
+    public void handshake_HappyPath()
+    {
+        // beacuse both CHH & SHH are ChannelInboundHandlers, we can't use the same EmbeddedChannel to handle them
+        InboundHandshakeHandler inboundHandshakeHandler = new InboundHandshakeHandler(new TestAuthenticator(true));
+        EmbeddedChannel inboundChannel = new EmbeddedChannel(inboundHandshakeHandler);
+
+        OutboundMessagingConnection imc = new OutboundMessagingConnection(connectionId, null, Optional.empty(), new AllowAllInternodeAuthenticator());
+        OutboundConnectionParams params = OutboundConnectionParams.builder()
+                                                                  .connectionId(connectionId)
+                                                                  .callback(imc::finishHandshake)
+                                                                  .mode(NettyFactory.Mode.MESSAGING)
+                                                                  .protocolVersion(MessagingService.current_version)
+                                                                  .coalescingStrategy(Optional.empty())
+                                                                  .build();
+        OutboundHandshakeHandler outboundHandshakeHandler = new OutboundHandshakeHandler(params);
+        EmbeddedChannel outboundChannel = new EmbeddedChannel(outboundHandshakeHandler);
+        Assert.assertEquals(1, outboundChannel.outboundMessages().size());
+
+        // move internode protocol Msg1 to the server's channel
+        Object o;
+        while ((o = outboundChannel.readOutbound()) != null)
+            inboundChannel.writeInbound(o);
+            Assert.assertEquals(1, inboundChannel.outboundMessages().size());
+
+        // move internode protocol Msg2 to the client's channel
+        while ((o = inboundChannel.readOutbound()) != null)
+            outboundChannel.writeInbound(o);
+        Assert.assertEquals(1, outboundChannel.outboundMessages().size());
+
+        // move internode protocol Msg3 to the server's channel
+        while ((o = outboundChannel.readOutbound()) != null)
+            inboundChannel.writeInbound(o);
+
+        Assert.assertEquals(READY, imc.getState());
+        Assert.assertEquals(MESSAGING_HANDSHAKE_COMPLETE, inboundHandshakeHandler.getState());
+    }
+
+    @Test
+    public void lotsOfMutations_NoCompression() throws IOException
+    {
+        lotsOfMutations(false);
+    }
+
+    @Test
+    public void lotsOfMutations_WithCompression() throws IOException
+    {
+        lotsOfMutations(true);
+    }
+
+    private void lotsOfMutations(boolean compress)
+    {
+        TestChannels channels = buildChannels(compress);
+        EmbeddedChannel outboundChannel = channels.outboundChannel;
+        EmbeddedChannel inboundChannel = channels.inboundChannel;
+
+        // now the actual test!
+        ByteBuffer buf = ByteBuffer.allocate(1 << 10);
+        byte[] bytes = "ThisIsA16CharStr".getBytes();
+        while (buf.remaining() > 0)
+            buf.put(bytes);
+
+        // write a bunch of messages to the channel
+        ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+        int count = 1024;
+        for (int i = 0; i < count; i++)
+        {
+            if (i % 2 == 0)
+            {
+                Mutation mutation = new RowUpdateBuilder(cfs1.metadata.get(), 0, "k")
+                                    .clustering("bytes")
+                                    .add("val", buf)
+                                    .build();
+
+                QueuedMessage msg = new QueuedMessage(mutation.createMessage(), i);
+                outboundChannel.writeAndFlush(msg);
+            }
+            else
+            {
+                outboundChannel.writeAndFlush(new QueuedMessage(new MessageOut<>(MessagingService.Verb.ECHO), i));
+            }
+        }
+        outboundChannel.flush();
+
+        // move the messages to the other channel
+        Object o;
+        while ((o = outboundChannel.readOutbound()) != null)
+            inboundChannel.writeInbound(o);
+
+        Assert.assertTrue(outboundChannel.outboundMessages().isEmpty());
+        Assert.assertFalse(inboundChannel.finishAndReleaseAll());
+    }
+
+    private TestChannels buildChannels(boolean compress)
+    {
+        OutboundConnectionParams params = OutboundConnectionParams.builder()
+                                                                  .connectionId(connectionId)
+                                                                  .callback(this::nop)
+                                                                  .mode(NettyFactory.Mode.MESSAGING)
+                                                                  .compress(compress)
+                                                                  .coalescingStrategy(Optional.empty())
+                                                                  .protocolVersion(MessagingService.current_version)
+                                                                  .build();
+        OutboundHandshakeHandler outboundHandshakeHandler = new OutboundHandshakeHandler(params);
+        EmbeddedChannel outboundChannel = new EmbeddedChannel(outboundHandshakeHandler);
+        OutboundMessagingConnection omc = new OutboundMessagingConnection(connectionId, null, Optional.empty(), new AllowAllInternodeAuthenticator());
+        omc.setTargetVersion(MESSAGING_VERSION);
+        outboundHandshakeHandler.setupPipeline(outboundChannel, MESSAGING_VERSION);
+
+        // remove the outbound handshake message from the outbound messages
+        outboundChannel.outboundMessages().clear();
+
+        InboundHandshakeHandler handler = new InboundHandshakeHandler(new TestAuthenticator(true));
+        EmbeddedChannel inboundChannel = new EmbeddedChannel(handler);
+        handler.setupMessagingPipeline(inboundChannel.pipeline(), REMOTE_ADDR.getAddress(), compress, MESSAGING_VERSION);
+
+        return new TestChannels(outboundChannel, inboundChannel);
+    }
+
+    private static class TestChannels
+    {
+        final EmbeddedChannel outboundChannel;
+        final EmbeddedChannel inboundChannel;
+
+        TestChannels(EmbeddedChannel outboundChannel, EmbeddedChannel inboundChannel)
+        {
+            this.outboundChannel = outboundChannel;
+            this.inboundChannel = inboundChannel;
+        }
+    }
+
+    private Void nop(OutboundHandshakeHandler.HandshakeResult handshakeResult)
+    {
+        // do nothing, really
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java b/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java
new file mode 100644
index 0000000..a3d646d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.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.net.async;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage;
+import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage;
+import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.assertEquals;
+
+public class HandshakeProtocolTest
+{
+    private ByteBuf buf;
+
+    @BeforeClass
+    public static void before()
+    {
+        // Kind of stupid, but the test trigger the initialization of the MessagingService class and that require
+        // DatabaseDescriptor to be configured ...
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (buf != null && buf.refCnt() > 0)
+            buf.release();
+    }
+
+    @Test
+    public void firstMessageTest() throws Exception
+    {
+        firstMessageTest(NettyFactory.Mode.MESSAGING, false);
+        firstMessageTest(NettyFactory.Mode.MESSAGING, true);
+        firstMessageTest(NettyFactory.Mode.STREAMING, false);
+        firstMessageTest(NettyFactory.Mode.STREAMING, true);
+    }
+
+    private void firstMessageTest(NettyFactory.Mode mode, boolean compression) throws Exception
+    {
+        FirstHandshakeMessage before = new FirstHandshakeMessage(MessagingService.current_version, mode, compression);
+        buf = before.encode(PooledByteBufAllocator.DEFAULT);
+        FirstHandshakeMessage after = FirstHandshakeMessage.maybeDecode(buf);
+        assertEquals(before, after);
+        assertEquals(before.hashCode(), after.hashCode());
+        Assert.assertFalse(before.equals(null));
+    }
+
+    @Test
+    public void secondMessageTest() throws Exception
+    {
+        SecondHandshakeMessage before = new SecondHandshakeMessage(MessagingService.current_version);
+        buf = before.encode(PooledByteBufAllocator.DEFAULT);
+        SecondHandshakeMessage after = SecondHandshakeMessage.maybeDecode(buf);
+        assertEquals(before, after);
+        assertEquals(before.hashCode(), after.hashCode());
+        Assert.assertFalse(before.equals(null));
+    }
+
+    @Test
+    public void thirdMessageTest() throws Exception
+    {
+        ThirdHandshakeMessage before = new ThirdHandshakeMessage(MessagingService.current_version, FBUtilities.getBroadcastAddress());
+        buf = before.encode(PooledByteBufAllocator.DEFAULT);
+        ThirdHandshakeMessage after = ThirdHandshakeMessage.maybeDecode(buf);
+        assertEquals(before, after);
+        assertEquals(before.hashCode(), after.hashCode());
+        Assert.assertFalse(before.equals(null));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java b/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
new file mode 100644
index 0000000..44dc469
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
@@ -0,0 +1,289 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufOutputStream;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import io.netty.handler.codec.compression.Lz4FrameDecoder;
+import io.netty.handler.codec.compression.Lz4FrameEncoder;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage;
+import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage;
+import org.apache.cassandra.net.async.InboundHandshakeHandler.State;
+
+import static org.apache.cassandra.net.async.NettyFactory.Mode.MESSAGING;
+
+public class InboundHandshakeHandlerTest
+{
+    private static final InetSocketAddress addr = new InetSocketAddress("127.0.0.1", 0);
+    private static final int MESSAGING_VERSION = MessagingService.current_version;
+    private static final int VERSION_30 = MessagingService.VERSION_30;
+
+    private InboundHandshakeHandler handler;
+    private EmbeddedChannel channel;
+    private ByteBuf buf;
+
+    @BeforeClass
+    public static void beforeClass()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Before
+    public void setUp()
+    {
+        TestAuthenticator authenticator = new TestAuthenticator(false);
+        handler = new InboundHandshakeHandler(authenticator);
+        channel = new EmbeddedChannel(handler);
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (buf != null)
+            buf.release();
+        channel.finishAndReleaseAll();
+    }
+
+    @Test
+    public void handleAuthenticate_Good()
+    {
+        handler = new InboundHandshakeHandler(new TestAuthenticator(true));
+        channel = new EmbeddedChannel(handler);
+        boolean result = handler.handleAuthenticate(addr, channel.pipeline().firstContext());
+        Assert.assertTrue(result);
+        Assert.assertTrue(channel.isOpen());
+    }
+
+    @Test
+    public void handleAuthenticate_Bad()
+    {
+        boolean result = handler.handleAuthenticate(addr, channel.pipeline().firstContext());
+        Assert.assertFalse(result);
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertFalse(channel.isActive());
+    }
+
+    @Test
+    public void handleAuthenticate_BadSocketAddr()
+    {
+        boolean result = handler.handleAuthenticate(new FakeSocketAddress(), channel.pipeline().firstContext());
+        Assert.assertFalse(result);
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertFalse(channel.isActive());
+    }
+
+    private static class FakeSocketAddress extends SocketAddress
+    {   }
+
+    @Test
+    public void decode_AlreadyFailed()
+    {
+        handler.setState(State.HANDSHAKE_FAIL);
+        buf = new FirstHandshakeMessage(MESSAGING_VERSION, MESSAGING, true).encode(PooledByteBufAllocator.DEFAULT);
+        handler.decode(channel.pipeline().firstContext(), buf, new ArrayList<>());
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertFalse(channel.isActive());
+        Assert.assertSame(State.HANDSHAKE_FAIL, handler.getState());
+    }
+
+    @Test
+    public void handleStart_NotEnoughInputBytes() throws IOException
+    {
+        ByteBuf buf = Unpooled.EMPTY_BUFFER;
+        State state = handler.handleStart(channel.pipeline().firstContext(), buf);
+        Assert.assertEquals(State.START, state);
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertTrue(channel.isActive());
+    }
+
+    @Test (expected = IOException.class)
+    public void handleStart_BadMagic() throws IOException
+    {
+        InboundHandshakeHandler handler = new InboundHandshakeHandler(new TestAuthenticator(false));
+        EmbeddedChannel channel = new EmbeddedChannel(handler);
+        buf = Unpooled.buffer(32, 32);
+
+        FirstHandshakeMessage first = new FirstHandshakeMessage(MESSAGING_VERSION,
+                                                                MESSAGING,
+                                                                true);
+
+        buf.writeInt(MessagingService.PROTOCOL_MAGIC << 2);
+        buf.writeInt(first.encodeFlags());
+        handler.handleStart(channel.pipeline().firstContext(), buf);
+    }
+
+    @Test
+    public void handleStart_VersionTooHigh() throws IOException
+    {
+        channel.eventLoop();
+        buf = new FirstHandshakeMessage(MESSAGING_VERSION + 1, MESSAGING, true).encode(PooledByteBufAllocator.DEFAULT);
+        State state = handler.handleStart(channel.pipeline().firstContext(), buf);
+        Assert.assertEquals(State.HANDSHAKE_FAIL, state);
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertFalse(channel.isActive());
+    }
+
+    @Test
+    public void handleStart_VersionLessThan3_0() throws IOException
+    {
+        buf = new FirstHandshakeMessage(VERSION_30 - 1, MESSAGING, true).encode(PooledByteBufAllocator.DEFAULT);
+        State state = handler.handleStart(channel.pipeline().firstContext(), buf);
+        Assert.assertEquals(State.HANDSHAKE_FAIL, state);
+
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertFalse(channel.isActive());
+    }
+
+    @Test
+    public void handleStart_HappyPath_Messaging() throws IOException
+    {
+        buf = new FirstHandshakeMessage(MESSAGING_VERSION, MESSAGING, true).encode(PooledByteBufAllocator.DEFAULT);
+        State state = handler.handleStart(channel.pipeline().firstContext(), buf);
+        Assert.assertEquals(State.AWAIT_MESSAGING_START_RESPONSE, state);
+        if (buf.refCnt() > 0)
+            buf.release();
+
+        buf = new ThirdHandshakeMessage(MESSAGING_VERSION, addr.getAddress()).encode(PooledByteBufAllocator.DEFAULT);
+        state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
+
+        Assert.assertEquals(State.MESSAGING_HANDSHAKE_COMPLETE, state);
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertTrue(channel.isActive());
+        Assert.assertFalse(channel.outboundMessages().isEmpty());
+        channel.releaseOutbound();
+    }
+
+    @Test
+    public void handleMessagingStartResponse_NotEnoughInputBytes() throws IOException
+    {
+        ByteBuf buf = Unpooled.EMPTY_BUFFER;
+        State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
+        Assert.assertEquals(State.AWAIT_MESSAGING_START_RESPONSE, state);
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertTrue(channel.isActive());
+    }
+
+    @Test
+    public void handleMessagingStartResponse_BadMaxVersion() throws IOException
+    {
+        buf = Unpooled.buffer(32, 32);
+        buf.writeInt(MESSAGING_VERSION + 1);
+        CompactEndpointSerializationHelper.serialize(addr.getAddress(), new ByteBufOutputStream(buf));
+        State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
+        Assert.assertEquals(State.HANDSHAKE_FAIL, state);
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertFalse(channel.isActive());
+    }
+
+    @Test
+    public void handleMessagingStartResponse_HappyPath() throws IOException
+    {
+        buf = Unpooled.buffer(32, 32);
+        buf.writeInt(MESSAGING_VERSION);
+        CompactEndpointSerializationHelper.serialize(addr.getAddress(), new ByteBufOutputStream(buf));
+        State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
+        Assert.assertEquals(State.MESSAGING_HANDSHAKE_COMPLETE, state);
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertTrue(channel.isActive());
+    }
+
+    @Test
+    public void setupPipeline_NoCompression()
+    {
+        ChannelPipeline pipeline = channel.pipeline();
+        Assert.assertNotNull(pipeline.get(InboundHandshakeHandler.class));
+
+        handler.setupMessagingPipeline(pipeline, addr.getAddress(), false, MESSAGING_VERSION);
+        Assert.assertNotNull(pipeline.get(MessageInHandler.class));
+        Assert.assertNull(pipeline.get(Lz4FrameDecoder.class));
+        Assert.assertNull(pipeline.get(Lz4FrameEncoder.class));
+        Assert.assertNull(pipeline.get(InboundHandshakeHandler.class));
+    }
+
+    @Test
+    public void setupPipeline_WithCompression()
+    {
+        ChannelPipeline pipeline = channel.pipeline();
+        Assert.assertNotNull(pipeline.get(InboundHandshakeHandler.class));
+
+        handler.setupMessagingPipeline(pipeline, addr.getAddress(), true, MESSAGING_VERSION);
+        Assert.assertNotNull(pipeline.get(MessageInHandler.class));
+        Assert.assertNotNull(pipeline.get(Lz4FrameDecoder.class));
+        Assert.assertNull(pipeline.get(Lz4FrameEncoder.class));
+        Assert.assertNull(pipeline.get(InboundHandshakeHandler.class));
+    }
+
+    @Test
+    public void failHandshake()
+    {
+        ChannelPromise future = channel.newPromise();
+        handler.setHandshakeTimeout(future);
+        Assert.assertFalse(future.isCancelled());
+        Assert.assertTrue(channel.isOpen());
+        handler.failHandshake(channel.pipeline().firstContext());
+        Assert.assertSame(State.HANDSHAKE_FAIL, handler.getState());
+        Assert.assertTrue(future.isCancelled());
+        Assert.assertFalse(channel.isOpen());
+    }
+
+    @Test
+    public void failHandshake_AlreadyConnected()
+    {
+        ChannelPromise future = channel.newPromise();
+        handler.setHandshakeTimeout(future);
+        Assert.assertFalse(future.isCancelled());
+        Assert.assertTrue(channel.isOpen());
+        handler.setState(State.MESSAGING_HANDSHAKE_COMPLETE);
+        handler.failHandshake(channel.pipeline().firstContext());
+        Assert.assertSame(State.MESSAGING_HANDSHAKE_COMPLETE, handler.getState());
+        Assert.assertTrue(channel.isOpen());
+    }
+
+    @Test
+    public void failHandshake_TaskIsCancelled()
+    {
+        ChannelPromise future = channel.newPromise();
+        future.cancel(false);
+        handler.setHandshakeTimeout(future);
+        handler.setState(State.AWAIT_MESSAGING_START_RESPONSE);
+        Assert.assertTrue(channel.isOpen());
+        handler.failHandshake(channel.pipeline().firstContext());
+        Assert.assertSame(State.AWAIT_MESSAGING_START_RESPONSE, handler.getState());
+        Assert.assertTrue(channel.isOpen());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java b/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java
new file mode 100644
index 0000000..bb82d2c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java
@@ -0,0 +1,242 @@
+/*
+ * 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.async;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+
+import com.google.common.base.Charsets;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufOutputStream;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.embedded.EmbeddedChannel;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.util.concurrent.Future;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.MessageInHandler.MessageHeader;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis;
+
+public class MessageInHandlerTest
+{
+    private static final InetSocketAddress addr = new InetSocketAddress("127.0.0.1", 0);
+    private static final int MSG_VERSION = MessagingService.current_version;
+
+    private static final int MSG_ID = 42;
+
+    private ByteBuf buf;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (buf != null && buf.refCnt() > 0)
+            buf.release();
+    }
+
+    @Test
+    public void decode_BadMagic() throws Exception
+    {
+        int len = MessageInHandler.FIRST_SECTION_BYTE_COUNT;
+        buf = Unpooled.buffer(len, len);
+        buf.writeInt(-1);
+        buf.writerIndex(len);
+
+        MessageInHandler handler = new MessageInHandler(addr.getAddress(), MSG_VERSION, null);
+        EmbeddedChannel channel = new EmbeddedChannel(handler);
+        Assert.assertTrue(channel.isOpen());
+        channel.writeInbound(buf);
+        Assert.assertFalse(channel.isOpen());
+    }
+
+    @Test
+    public void decode_HappyPath_NoParameters() throws Exception
+    {
+        MessageInWrapper result = decode_HappyPath(Collections.emptyMap());
+        Assert.assertTrue(result.messageIn.parameters.isEmpty());
+    }
+
+    @Test
+    public void decode_HappyPath_WithParameters() throws Exception
+    {
+        Map<String, byte[]> parameters = new HashMap<>();
+        parameters.put("p1", "val1".getBytes(Charsets.UTF_8));
+        parameters.put("p2", "val2".getBytes(Charsets.UTF_8));
+        MessageInWrapper result = decode_HappyPath(parameters);
+        Assert.assertEquals(2, result.messageIn.parameters.size());
+    }
+
+    private MessageInWrapper decode_HappyPath(Map<String, byte[]> parameters) throws Exception
+    {
+        MessageOut msgOut = new MessageOut(MessagingService.Verb.ECHO);
+        for (Map.Entry<String, byte[]> param : parameters.entrySet())
+            msgOut = msgOut.withParameter(param.getKey(), param.getValue());
+        serialize(msgOut);
+
+        MessageInWrapper wrapper = new MessageInWrapper();
+        MessageInHandler handler = new MessageInHandler(addr.getAddress(), MSG_VERSION, wrapper.messageConsumer);
+        List<Object> out = new ArrayList<>();
+        handler.decode(null, buf, out);
+
+        Assert.assertNotNull(wrapper.messageIn);
+        Assert.assertEquals(MSG_ID, wrapper.id);
+        Assert.assertEquals(msgOut.from, wrapper.messageIn.from);
+        Assert.assertEquals(msgOut.verb, wrapper.messageIn.verb);
+        Assert.assertTrue(out.isEmpty());
+
+        return wrapper;
+    }
+
+    private void serialize(MessageOut msgOut) throws IOException
+    {
+        buf = Unpooled.buffer(1024, 1024); // 1k should be enough for everybody!
+        buf.writeInt(MessagingService.PROTOCOL_MAGIC);
+        buf.writeInt(MSG_ID); // this is the id
+        buf.writeInt((int) NanoTimeToCurrentTimeMillis.convert(System.nanoTime()));
+
+        msgOut.serialize(new ByteBufDataOutputPlus(buf), MSG_VERSION);
+    }
+
+    @Test
+    public void decode_WithHalfReceivedParameters() throws Exception
+    {
+        MessageOut msgOut = new MessageOut(MessagingService.Verb.ECHO);
+        msgOut = msgOut.withParameter("p3", "val1".getBytes(Charsets.UTF_8));
+
+        serialize(msgOut);
+
+        // move the write index pointer back a few bytes to simulate like the full bytes are not present.
+        // yeah, it's lame, but it tests the basics of what is happening during the deserialiization
+        int originalWriterIndex = buf.writerIndex();
+        buf.writerIndex(originalWriterIndex - 6);
+
+        MessageInWrapper wrapper = new MessageInWrapper();
+        MessageInHandler handler = new MessageInHandler(addr.getAddress(), MSG_VERSION, wrapper.messageConsumer);
+        List<Object> out = new ArrayList<>();
+        handler.decode(null, buf, out);
+
+        Assert.assertNull(wrapper.messageIn);
+
+        MessageHeader header = handler.getMessageHeader();
+        Assert.assertEquals(MSG_ID, header.messageId);
+        Assert.assertEquals(msgOut.verb, header.verb);
+        Assert.assertEquals(msgOut.from, header.from);
+        Assert.assertTrue(out.isEmpty());
+
+        // now, set the writer index back to the original value to pretend that we actually got more bytes in
+        buf.writerIndex(originalWriterIndex);
+        handler.decode(null, buf, out);
+        Assert.assertNotNull(wrapper.messageIn);
+        Assert.assertTrue(out.isEmpty());
+    }
+
+    @Test
+    public void canReadNextParam_HappyPath() throws IOException
+    {
+        buildParamBuf(13);
+        Assert.assertTrue(MessageInHandler.canReadNextParam(buf));
+    }
+
+    @Test
+    public void canReadNextParam_OnlyFirstByte() throws IOException
+    {
+        buildParamBuf(13);
+        buf.writerIndex(1);
+        Assert.assertFalse(MessageInHandler.canReadNextParam(buf));
+    }
+
+    @Test
+    public void canReadNextParam_PartialUTF() throws IOException
+    {
+        buildParamBuf(13);
+        buf.writerIndex(5);
+        Assert.assertFalse(MessageInHandler.canReadNextParam(buf));
+    }
+
+    @Test
+    public void canReadNextParam_TruncatedValueLength() throws IOException
+    {
+        buildParamBuf(13);
+        buf.writerIndex(buf.writerIndex() - 13 - 2);
+        Assert.assertFalse(MessageInHandler.canReadNextParam(buf));
+    }
+
+    @Test
+    public void canReadNextParam_MissingLastBytes() throws IOException
+    {
+        buildParamBuf(13);
+        buf.writerIndex(buf.writerIndex() - 2);
+        Assert.assertFalse(MessageInHandler.canReadNextParam(buf));
+    }
+
+    private void buildParamBuf(int valueLength) throws IOException
+    {
+        buf = Unpooled.buffer(1024, 1024); // 1k should be enough for everybody!
+        ByteBufDataOutputPlus output = new ByteBufDataOutputPlus(buf);
+        output.writeUTF("name");
+        byte[] array = new byte[valueLength];
+        output.writeInt(array.length);
+        output.write(array);
+    }
+
+    @Test
+    public void exceptionHandled()
+    {
+        MessageInHandler handler = new MessageInHandler(addr.getAddress(), MSG_VERSION, null);
+        EmbeddedChannel channel = new EmbeddedChannel(handler);
+        Assert.assertTrue(channel.isOpen());
+        handler.exceptionCaught(channel.pipeline().firstContext(), new EOFException());
+        Assert.assertFalse(channel.isOpen());
+    }
+
+    private static class MessageInWrapper
+    {
+        MessageIn messageIn;
+        int id;
+
+        final BiConsumer<MessageIn, Integer> messageConsumer = (messageIn, integer) ->
+        {
+            this.messageIn = messageIn;
+            this.id = integer;
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java b/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java
new file mode 100644
index 0000000..566dfdb
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java
@@ -0,0 +1,289 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.sun.org.apache.bcel.internal.generic.DDIV;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.DefaultChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import io.netty.handler.codec.UnsupportedMessageTypeException;
+import io.netty.handler.timeout.IdleStateEvent;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.UUIDGen;
+
+public class MessageOutHandlerTest
+{
+    private static final int MESSAGING_VERSION = MessagingService.current_version;
+
+    private ChannelWriter channelWriter;
+    private EmbeddedChannel channel;
+    private MessageOutHandler handler;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+        DatabaseDescriptor.createAllDirectories();
+    }
+
+    @Before
+    public void setup()
+    {
+        setup(MessageOutHandler.AUTO_FLUSH_THRESHOLD);
+    }
+
+    private void setup(int flushThreshold)
+    {
+        OutboundConnectionIdentifier connectionId = OutboundConnectionIdentifier.small(new InetSocketAddress("127.0.0.1", 0),
+                                                                                       new InetSocketAddress("127.0.0.2", 0));
+        OutboundMessagingConnection omc = new NonSendingOutboundMessagingConnection(connectionId, null, Optional.empty());
+        channel = new EmbeddedChannel();
+        channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty());
+        handler = new MessageOutHandler(connectionId, MESSAGING_VERSION, channelWriter, () -> null, flushThreshold);
+        channel.pipeline().addLast(handler);
+    }
+
+    @Test
+    public void write_NoFlush() throws ExecutionException, InterruptedException, TimeoutException
+    {
+        MessageOut message = new MessageOut(MessagingService.Verb.ECHO);
+        ChannelFuture future = channel.write(new QueuedMessage(message, 42));
+        Assert.assertTrue(!future.isDone());
+        Assert.assertFalse(channel.releaseOutbound());
+    }
+
+    @Test
+    public void write_WithFlush() throws ExecutionException, InterruptedException, TimeoutException
+    {
+        setup(1);
+        MessageOut message = new MessageOut(MessagingService.Verb.ECHO);
+        ChannelFuture future = channel.write(new QueuedMessage(message, 42));
+        Assert.assertTrue(future.isSuccess());
+        Assert.assertTrue(channel.releaseOutbound());
+    }
+
+    @Test
+    public void serializeMessage() throws IOException
+    {
+        channelWriter.pendingMessageCount.set(1);
+        QueuedMessage msg = new QueuedMessage(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), 1);
+        ChannelFuture future = channel.writeAndFlush(msg);
+
+        Assert.assertTrue(future.isSuccess());
+        Assert.assertTrue(1 <= channel.outboundMessages().size());
+        Assert.assertTrue(channel.releaseOutbound());
+    }
+
+    @Test
+    public void wrongMessageType()
+    {
+        ChannelPromise promise = new DefaultChannelPromise(channel);
+        Assert.assertFalse(handler.isMessageValid("this is the wrong message type", promise));
+
+        Assert.assertFalse(promise.isSuccess());
+        Assert.assertNotNull(promise.cause());
+        Assert.assertSame(UnsupportedMessageTypeException.class, promise.cause().getClass());
+    }
+
+    @Test
+    public void unexpiredMessage()
+    {
+        QueuedMessage msg = new QueuedMessage(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), 1);
+        ChannelPromise promise = new DefaultChannelPromise(channel);
+        Assert.assertTrue(handler.isMessageValid(msg, promise));
+
+        // we won't know if it was successful yet, but we'll know if it's a failure because cause will be set
+        Assert.assertNull(promise.cause());
+    }
+
+    @Test
+    public void expiredMessage()
+    {
+        QueuedMessage msg = new QueuedMessage(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), 1, 0, true, true);
+        ChannelPromise promise = new DefaultChannelPromise(channel);
+        Assert.assertFalse(handler.isMessageValid(msg, promise));
+
+        Assert.assertFalse(promise.isSuccess());
+        Assert.assertNotNull(promise.cause());
+        Assert.assertSame(ExpiredException.class, promise.cause().getClass());
+        Assert.assertTrue(channel.outboundMessages().isEmpty());
+    }
+
+    @Test
+    public void write_MessageTooLarge()
+    {
+        write_BadMessageSize(Integer.MAX_VALUE + 1);
+    }
+
+    @Test
+    public void write_MessageSizeIsBananas()
+    {
+        write_BadMessageSize(Integer.MIN_VALUE + 10000);
+    }
+
+    private void write_BadMessageSize(long size)
+    {
+        IVersionedSerializer<Object> serializer = new IVersionedSerializer<Object>()
+        {
+            public void serialize(Object o, DataOutputPlus out, int version)
+            {   }
+
+            public Object deserialize(DataInputPlus in, int version)
+            {
+                return null;
+            }
+
+            public long serializedSize(Object o, int version)
+            {
+                return size;
+            }
+        };
+        MessageOut message = new MessageOut(MessagingService.Verb.UNUSED_5, "payload", serializer);
+        ChannelFuture future = channel.write(new QueuedMessage(message, 42));
+        Throwable t = future.cause();
+        Assert.assertNotNull(t);
+        Assert.assertSame(IllegalStateException.class, t.getClass());
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertFalse(channel.releaseOutbound());
+    }
+
+    @Test
+    public void writeForceExceptionPath()
+    {
+        IVersionedSerializer<Object> serializer = new IVersionedSerializer<Object>()
+        {
+            public void serialize(Object o, DataOutputPlus out, int version)
+            {
+                throw new RuntimeException("this exception is part of the test - DON'T PANIC");
+            }
+
+            public Object deserialize(DataInputPlus in, int version)
+            {
+                return null;
+            }
+
+            public long serializedSize(Object o, int version)
+            {
+                return 42;
+            }
+        };
+        MessageOut message = new MessageOut(MessagingService.Verb.UNUSED_5, "payload", serializer);
+        ChannelFuture future = channel.write(new QueuedMessage(message, 42));
+        Throwable t = future.cause();
+        Assert.assertNotNull(t);
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertFalse(channel.releaseOutbound());
+    }
+
+    @Test
+    public void captureTracingInfo_ForceException()
+    {
+        MessageOut message = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
+                             .withParameter(Tracing.TRACE_HEADER, new byte[9]);
+        handler.captureTracingInfo(new QueuedMessage(message, 42));
+    }
+
+    @Test
+    public void captureTracingInfo_UnknownSession()
+    {
+        UUID uuid = UUID.randomUUID();
+        MessageOut message = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
+                             .withParameter(Tracing.TRACE_HEADER, UUIDGen.decompose(uuid));
+        handler.captureTracingInfo(new QueuedMessage(message, 42));
+    }
+
+    @Test
+    public void captureTracingInfo_KnownSession()
+    {
+        Tracing.instance.newSession(new HashMap<>());
+        MessageOut message = new MessageOut(MessagingService.Verb.REQUEST_RESPONSE);
+        handler.captureTracingInfo(new QueuedMessage(message, 42));
+    }
+
+    @Test
+    public void userEventTriggered_RandomObject()
+    {
+        Assert.assertTrue(channel.isOpen());
+        ChannelUserEventSender sender = new ChannelUserEventSender();
+        channel.pipeline().addFirst(sender);
+        sender.sendEvent("ThisIsAFakeEvent");
+        Assert.assertTrue(channel.isOpen());
+    }
+
+    @Test
+    public void userEventTriggered_Idle_NoPendingBytes()
+    {
+        Assert.assertTrue(channel.isOpen());
+        ChannelUserEventSender sender = new ChannelUserEventSender();
+        channel.pipeline().addFirst(sender);
+        sender.sendEvent(IdleStateEvent.WRITER_IDLE_STATE_EVENT);
+        Assert.assertTrue(channel.isOpen());
+    }
+
+    @Test
+    public void userEventTriggered_Idle_WithPendingBytes()
+    {
+        Assert.assertTrue(channel.isOpen());
+        ChannelUserEventSender sender = new ChannelUserEventSender();
+        channel.pipeline().addFirst(sender);
+
+        MessageOut message = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE);
+        channel.writeOutbound(new QueuedMessage(message, 42));
+        sender.sendEvent(IdleStateEvent.WRITER_IDLE_STATE_EVENT);
+        Assert.assertFalse(channel.isOpen());
+    }
+
+    private static class ChannelUserEventSender extends ChannelOutboundHandlerAdapter
+    {
+        private ChannelHandlerContext ctx;
+
+        @Override
+        public void handlerAdded(final ChannelHandlerContext ctx) throws Exception
+        {
+            this.ctx = ctx;
+        }
+
+        private void sendEvent(Object event)
+        {
+            ctx.fireUserEventTriggered(event);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java b/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java
new file mode 100644
index 0000000..c4cc7e6
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java
@@ -0,0 +1,300 @@
+/*
+ * 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.async;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Optional;
+
+import com.google.common.net.InetAddresses;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.DefaultEventLoop;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollServerSocketChannel;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.group.DefaultChannelGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.ssl.SslHandler;
+import io.netty.util.concurrent.GlobalEventExecutor;
+import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
+import org.apache.cassandra.auth.IInternodeAuthenticator;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions.InternodeEncryption;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.NettyFactory.InboundInitializer;
+import org.apache.cassandra.net.async.NettyFactory.OutboundInitializer;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.NativeLibrary;
+
+public class NettyFactoryTest
+{
+    private static final InetSocketAddress LOCAL_ADDR = new InetSocketAddress("127.0.0.1", 9876);
+    private static final InetSocketAddress REMOTE_ADDR = new InetSocketAddress("127.0.0.2", 9876);
+    private static final int receiveBufferSize = 1 << 16;
+    private static final IInternodeAuthenticator AUTHENTICATOR = new AllowAllInternodeAuthenticator();
+
+    private ChannelGroup channelGroup;
+    private NettyFactory factory;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Before
+    public void setUp()
+    {
+        channelGroup = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (factory != null)
+            factory.close();
+    }
+
+    @Test
+    public void createServerChannel_Epoll()
+    {
+        Channel inboundChannel = createServerChannel(true);
+        if (inboundChannel == null)
+            return;
+        Assert.assertEquals(EpollServerSocketChannel.class, inboundChannel.getClass());
+        inboundChannel.close();
+    }
+
+    private Channel createServerChannel(boolean useEpoll)
+    {
+        InboundInitializer inboundInitializer = new InboundInitializer(AUTHENTICATOR, null, channelGroup);
+        factory = new NettyFactory(useEpoll);
+
+        try
+        {
+            return factory.createInboundChannel(LOCAL_ADDR, inboundInitializer, receiveBufferSize);
+        }
+        catch (Exception e)
+        {
+            if (NativeLibrary.osType == NativeLibrary.OSType.LINUX)
+                throw e;
+
+            return null;
+        }
+    }
+
+    @Test
+    public void createServerChannel_Nio()
+    {
+        Channel inboundChannel = createServerChannel(false);
+        Assert.assertNotNull("we should always be able to get a NIO channel", inboundChannel);
+        Assert.assertEquals(NioServerSocketChannel.class, inboundChannel.getClass());
+        inboundChannel.close();
+    }
+
+    @Test(expected = ConfigurationException.class)
+    public void createServerChannel_SecondAttemptToBind()
+    {
+        Channel inboundChannel = null;
+        try
+        {
+            InetSocketAddress addr = new InetSocketAddress("127.0.0.1", 9876);
+            InboundInitializer inboundInitializer = new InboundInitializer(AUTHENTICATOR, null, channelGroup);
+            inboundChannel = NettyFactory.instance.createInboundChannel(LOCAL_ADDR, inboundInitializer, receiveBufferSize);
+            NettyFactory.instance.createInboundChannel(LOCAL_ADDR, inboundInitializer, receiveBufferSize);
+        }
+        finally
+        {
+            if (inboundChannel != null)
+                inboundChannel.close();
+        }
+    }
+
+    @Test(expected = ConfigurationException.class)
+    public void createServerChannel_UnbindableAddress()
+    {
+        InetSocketAddress addr = new InetSocketAddress("1.1.1.1", 9876);
+        InboundInitializer inboundInitializer = new InboundInitializer(AUTHENTICATOR, null, channelGroup);
+        NettyFactory.instance.createInboundChannel(addr, inboundInitializer, receiveBufferSize);
+    }
+
+    @Test
+    public void deterineAcceptGroupSize()
+    {
+        Assert.assertEquals(1, NettyFactory.determineAcceptGroupSize(InternodeEncryption.none));
+        Assert.assertEquals(1, NettyFactory.determineAcceptGroupSize(InternodeEncryption.all));
+        Assert.assertEquals(2, NettyFactory.determineAcceptGroupSize(InternodeEncryption.rack));
+        Assert.assertEquals(2, NettyFactory.determineAcceptGroupSize(InternodeEncryption.dc));
+
+        InetAddress originalBroadcastAddr = FBUtilities.getBroadcastAddress();
+        try
+        {
+            FBUtilities.setBroadcastInetAddress(InetAddresses.increment(FBUtilities.getLocalAddress()));
+            DatabaseDescriptor.setListenOnBroadcastAddress(true);
+
+            Assert.assertEquals(2, NettyFactory.determineAcceptGroupSize(InternodeEncryption.none));
+            Assert.assertEquals(2, NettyFactory.determineAcceptGroupSize(InternodeEncryption.all));
+            Assert.assertEquals(4, NettyFactory.determineAcceptGroupSize(InternodeEncryption.rack));
+            Assert.assertEquals(4, NettyFactory.determineAcceptGroupSize(InternodeEncryption.dc));
+        }
+        finally
+        {
+            FBUtilities.setBroadcastInetAddress(originalBroadcastAddr);
+            DatabaseDescriptor.setListenOnBroadcastAddress(false);
+        }
+    }
+
+    @Test
+    public void getEventLoopGroup_EpollWithIoRatioBoost()
+    {
+        getEventLoopGroup_Epoll(true);
+    }
+
+    private EpollEventLoopGroup getEventLoopGroup_Epoll(boolean ioBoost)
+    {
+        EventLoopGroup eventLoopGroup;
+        try
+        {
+            eventLoopGroup = NettyFactory.getEventLoopGroup(true, 1, "testEventLoopGroup", ioBoost);
+        }
+        catch (Exception e)
+        {
+            if (NativeLibrary.osType == NativeLibrary.OSType.LINUX)
+                throw e;
+
+            // ignore as epoll is only available on linux platforms, so don't fail the test on other OSes
+            return null;
+        }
+
+        Assert.assertTrue(eventLoopGroup instanceof EpollEventLoopGroup);
+        return (EpollEventLoopGroup) eventLoopGroup;
+    }
+
+    @Test
+    public void getEventLoopGroup_EpollWithoutIoRatioBoost()
+    {
+        getEventLoopGroup_Epoll(false);
+    }
+
+    @Test
+    public void getEventLoopGroup_NioWithoutIoRatioBoost()
+    {
+        getEventLoopGroup_Nio(true);
+    }
+
+    private NioEventLoopGroup getEventLoopGroup_Nio(boolean ioBoost)
+    {
+        EventLoopGroup eventLoopGroup = NettyFactory.getEventLoopGroup(false, 1, "testEventLoopGroup", ioBoost);
+        Assert.assertTrue(eventLoopGroup instanceof NioEventLoopGroup);
+        return (NioEventLoopGroup) eventLoopGroup;
+    }
+
+    @Test
+    public void getEventLoopGroup_NioWithIoRatioBoost()
+    {
+        getEventLoopGroup_Nio(true);
+    }
+
+    @Test
+    public void createOutboundBootstrap_Epoll()
+    {
+        Bootstrap bootstrap = createOutboundBootstrap(true);
+        Assert.assertEquals(EpollEventLoopGroup.class, bootstrap.config().group().getClass());
+    }
+
+    private Bootstrap createOutboundBootstrap(boolean useEpoll)
+    {
+        factory = new NettyFactory(useEpoll);
+        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.gossip(LOCAL_ADDR, REMOTE_ADDR);
+        OutboundConnectionParams params = OutboundConnectionParams.builder()
+                                                                  .connectionId(id)
+                                                                  .coalescingStrategy(Optional.empty())
+                                                                  .protocolVersion(MessagingService.current_version)
+                                                                  .build();
+        return factory.createOutboundBootstrap(params);
+    }
+
+    @Test
+    public void createOutboundBootstrap_Nio()
+    {
+        Bootstrap bootstrap = createOutboundBootstrap(false);
+        Assert.assertEquals(NioEventLoopGroup.class, bootstrap.config().group().getClass());
+    }
+
+    @Test
+    public void createInboundInitializer_WithoutSsl() throws Exception
+    {
+        InboundInitializer initializer = new InboundInitializer(AUTHENTICATOR, null, channelGroup);
+        NioSocketChannel channel = new NioSocketChannel();
+        initializer.initChannel(channel);
+        Assert.assertNull(channel.pipeline().get(SslHandler.class));
+    }
+
+    private ServerEncryptionOptions encOptions()
+    {
+        ServerEncryptionOptions encryptionOptions;
+        encryptionOptions = new ServerEncryptionOptions();
+        encryptionOptions.keystore = "test/conf/cassandra_ssl_test.keystore";
+        encryptionOptions.keystore_password = "cassandra";
+        encryptionOptions.truststore = "test/conf/cassandra_ssl_test.truststore";
+        encryptionOptions.truststore_password = "cassandra";
+        encryptionOptions.require_client_auth = false;
+        encryptionOptions.cipher_suites = new String[] {"TLS_RSA_WITH_AES_128_CBC_SHA"};
+        return encryptionOptions;
+    }
+    @Test
+    public void createInboundInitializer_WithSsl() throws Exception
+    {
+        ServerEncryptionOptions encryptionOptions = encOptions();
+        InboundInitializer initializer = new InboundInitializer(AUTHENTICATOR, encryptionOptions, channelGroup);
+        NioSocketChannel channel = new NioSocketChannel();
+        Assert.assertNull(channel.pipeline().get(SslHandler.class));
+        initializer.initChannel(channel);
+        Assert.assertNotNull(channel.pipeline().get(SslHandler.class));
+    }
+
+    @Test
+    public void createOutboundInitializer_WithSsl() throws Exception
+    {
+        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.gossip(LOCAL_ADDR, REMOTE_ADDR);
+        OutboundConnectionParams params = OutboundConnectionParams.builder()
+                                                                  .connectionId(id)
+                                                                  .encryptionOptions(encOptions())
+                                                                  .protocolVersion(MessagingService.current_version)
+                                                                  .build();
+        OutboundInitializer outboundInitializer = new OutboundInitializer(params);
+        NioSocketChannel channel = new NioSocketChannel();
+        Assert.assertNull(channel.pipeline().get(SslHandler.class));
+        outboundInitializer.initChannel(channel);
+        Assert.assertNotNull(channel.pipeline().get(SslHandler.class));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/NonSendingOutboundMessagingConnection.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/NonSendingOutboundMessagingConnection.java b/test/unit/org/apache/cassandra/net/async/NonSendingOutboundMessagingConnection.java
new file mode 100644
index 0000000..b0b15b8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/NonSendingOutboundMessagingConnection.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.net.async;
+
+import java.util.Optional;
+
+import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
+import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.utils.CoalescingStrategies;
+
+class NonSendingOutboundMessagingConnection extends OutboundMessagingConnection
+{
+    boolean sendMessageInvoked;
+
+    NonSendingOutboundMessagingConnection(OutboundConnectionIdentifier connectionId, EncryptionOptions.ServerEncryptionOptions encryptionOptions, Optional<CoalescingStrategies.CoalescingStrategy> coalescingStrategy)
+    {
+        super(connectionId, encryptionOptions, coalescingStrategy, new AllowAllInternodeAuthenticator());
+    }
+
+    @Override
+    boolean sendMessage(QueuedMessage queuedMessage)
+    {
+        sendMessageInvoked = true;
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java b/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java
new file mode 100644
index 0000000..0ce4968
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.async;
+
+import org.junit.Test;
+
+public class OutboundConnectionParamsTest
+{
+    @Test (expected = IllegalArgumentException.class)
+    public void build_SendSizeLessThanZero()
+    {
+        OutboundConnectionParams.builder().sendBufferSize(-1).build();
+    }
+
+    @Test (expected = IllegalArgumentException.class)
+    public void build_SendSizeHuge()
+    {
+        OutboundConnectionParams.builder().sendBufferSize(1 << 30).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java b/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java
new file mode 100644
index 0000000..f8bfab1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java
@@ -0,0 +1,209 @@
+/*
+ * 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.async;
+
+import java.net.InetSocketAddress;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.embedded.EmbeddedChannel;
+import io.netty.handler.codec.compression.Lz4FrameDecoder;
+import io.netty.handler.codec.compression.Lz4FrameEncoder;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage;
+import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult;
+
+import static org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult.UNKNOWN_PROTOCOL_VERSION;
+
+public class OutboundHandshakeHandlerTest
+{
+    private static final int MESSAGING_VERSION = MessagingService.current_version;
+    private static final InetSocketAddress localAddr = new InetSocketAddress("127.0.0.1", 0);
+    private static final InetSocketAddress remoteAddr = new InetSocketAddress("127.0.0.2", 0);
+    private static final String HANDLER_NAME = "clientHandshakeHandler";
+
+    private EmbeddedChannel channel;
+    private OutboundHandshakeHandler handler;
+    private OutboundConnectionIdentifier connectionId;
+    private OutboundConnectionParams params;
+    private CallbackHandler callbackHandler;
+    private ByteBuf buf;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Before
+    public void setup()
+    {
+        channel = new EmbeddedChannel(new ChannelOutboundHandlerAdapter());
+        connectionId = OutboundConnectionIdentifier.small(localAddr, remoteAddr);
+        callbackHandler = new CallbackHandler();
+        params = OutboundConnectionParams.builder()
+                                         .connectionId(connectionId)
+                                         .callback(handshakeResult -> callbackHandler.receive(handshakeResult))
+                                         .mode(NettyFactory.Mode.MESSAGING)
+                                         .protocolVersion(MessagingService.current_version)
+                                         .coalescingStrategy(Optional.empty())
+                                         .build();
+        handler = new OutboundHandshakeHandler(params);
+        channel.pipeline().addFirst(HANDLER_NAME, handler);
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (buf != null && buf.refCnt() > 0)
+            buf.release();
+        Assert.assertFalse(channel.finishAndReleaseAll());
+    }
+
+    @Test
+    public void decode_SmallInput() throws Exception
+    {
+        buf = Unpooled.buffer(2, 2);
+        List<Object> out = new LinkedList<>();
+        handler.decode(channel.pipeline().firstContext(), buf, out);
+        Assert.assertEquals(0, buf.readerIndex());
+        Assert.assertTrue(out.isEmpty());
+    }
+
+    @Test
+    public void decode_HappyPath() throws Exception
+    {
+        buf = new SecondHandshakeMessage(MESSAGING_VERSION).encode(PooledByteBufAllocator.DEFAULT);
+        channel.writeInbound(buf);
+        Assert.assertEquals(1, channel.outboundMessages().size());
+        Assert.assertTrue(channel.isOpen());
+        Assert.assertTrue(channel.releaseOutbound()); // throw away any responses from decode()
+
+        Assert.assertEquals(MESSAGING_VERSION, callbackHandler.result.negotiatedMessagingVersion);
+        Assert.assertEquals(HandshakeResult.Outcome.SUCCESS, callbackHandler.result.outcome);
+    }
+
+    @Test
+    public void decode_HappyPathThrowsException() throws Exception
+    {
+        callbackHandler.failOnCallback = true;
+        buf = new SecondHandshakeMessage(MESSAGING_VERSION).encode(PooledByteBufAllocator.DEFAULT);
+        channel.writeInbound(buf);
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertEquals(1, channel.outboundMessages().size());
+        Assert.assertTrue(channel.releaseOutbound()); // throw away any responses from decode()
+
+        Assert.assertEquals(UNKNOWN_PROTOCOL_VERSION, callbackHandler.result.negotiatedMessagingVersion);
+        Assert.assertEquals(HandshakeResult.Outcome.NEGOTIATION_FAILURE, callbackHandler.result.outcome);
+    }
+
+    @Test
+    public void decode_ReceivedLowerMsgVersion() throws Exception
+    {
+        int msgVersion = MESSAGING_VERSION - 1;
+        buf = new SecondHandshakeMessage(msgVersion).encode(PooledByteBufAllocator.DEFAULT);
+        channel.writeInbound(buf);
+        Assert.assertTrue(channel.inboundMessages().isEmpty());
+
+        Assert.assertEquals(msgVersion, callbackHandler.result.negotiatedMessagingVersion);
+        Assert.assertEquals(HandshakeResult.Outcome.DISCONNECT, callbackHandler.result.outcome);
+        Assert.assertFalse(channel.isOpen());
+        Assert.assertTrue(channel.outboundMessages().isEmpty());
+    }
+
+    @Test
+    public void decode_ReceivedHigherMsgVersion() throws Exception
+    {
+        int msgVersion = MESSAGING_VERSION - 1;
+        channel.pipeline().remove(HANDLER_NAME);
+        params = OutboundConnectionParams.builder()
+                                         .connectionId(connectionId)
+                                         .callback(handshakeResult -> callbackHandler.receive(handshakeResult))
+                                         .mode(NettyFactory.Mode.MESSAGING)
+                                         .protocolVersion(msgVersion)
+                                         .coalescingStrategy(Optional.empty())
+                                         .build();
+        handler = new OutboundHandshakeHandler(params);
+        channel.pipeline().addFirst(HANDLER_NAME, handler);
+        buf = new SecondHandshakeMessage(MESSAGING_VERSION).encode(PooledByteBufAllocator.DEFAULT);
+        channel.writeInbound(buf);
+
+        Assert.assertEquals(MESSAGING_VERSION, callbackHandler.result.negotiatedMessagingVersion);
+        Assert.assertEquals(HandshakeResult.Outcome.DISCONNECT, callbackHandler.result.outcome);
+    }
+
+    @Test
+    public void setupPipeline_WithCompression()
+    {
+        EmbeddedChannel chan = new EmbeddedChannel(new ChannelOutboundHandlerAdapter());
+        ChannelPipeline pipeline =  chan.pipeline();
+        params = OutboundConnectionParams.builder(params).compress(true).protocolVersion(MessagingService.current_version).build();
+        handler = new OutboundHandshakeHandler(params);
+        pipeline.addFirst(handler);
+        handler.setupPipeline(chan, MESSAGING_VERSION);
+        Assert.assertNotNull(pipeline.get(Lz4FrameEncoder.class));
+        Assert.assertNull(pipeline.get(Lz4FrameDecoder.class));
+        Assert.assertNotNull(pipeline.get(MessageOutHandler.class));
+    }
+
+    @Test
+    public void setupPipeline_NoCompression()
+    {
+        EmbeddedChannel chan = new EmbeddedChannel(new ChannelOutboundHandlerAdapter());
+        ChannelPipeline pipeline =  chan.pipeline();
+        params = OutboundConnectionParams.builder(params).compress(false).protocolVersion(MessagingService.current_version).build();
+        handler = new OutboundHandshakeHandler(params);
+        pipeline.addFirst(handler);
+        handler.setupPipeline(chan, MESSAGING_VERSION);
+        Assert.assertNull(pipeline.get(Lz4FrameEncoder.class));
+        Assert.assertNull(pipeline.get(Lz4FrameDecoder.class));
+        Assert.assertNotNull(pipeline.get(MessageOutHandler.class));
+    }
+
+    private static class CallbackHandler
+    {
+        boolean failOnCallback;
+        HandshakeResult result;
+
+        Void receive(HandshakeResult handshakeResult)
+        {
+            if (failOnCallback)
+            {
+                // only fail the first callback
+                failOnCallback = false;
+                throw new RuntimeException("this exception is expected in the test - DON'T PANIC");
+            }
+            result = handshakeResult;
+            return null;
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[05/11] cassandra git commit: switch internode messaging to netty

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
new file mode 100644
index 0000000..772e47d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
@@ -0,0 +1,519 @@
+/*
+ * 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.async;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.SSLHandshakeException;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
+import org.apache.cassandra.auth.IInternodeAuthenticator;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.MessagingServiceTest;
+import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult;
+import org.apache.cassandra.net.async.OutboundMessagingConnection.State;
+
+import static org.apache.cassandra.net.MessagingService.Verb.ECHO;
+import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.CLOSED;
+import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.CREATING_CHANNEL;
+import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.NOT_READY;
+import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.READY;
+
+public class OutboundMessagingConnectionTest
+{
+    private static final InetSocketAddress LOCAL_ADDR = new InetSocketAddress("127.0.0.1", 9998);
+    private static final InetSocketAddress REMOTE_ADDR = new InetSocketAddress("127.0.0.2", 9999);
+    private static final InetSocketAddress RECONNECT_ADDR = new InetSocketAddress("127.0.0.3", 9999);
+    private static final int MESSAGING_VERSION = MessagingService.current_version;
+
+    private OutboundConnectionIdentifier connectionId;
+    private OutboundMessagingConnection omc;
+    private EmbeddedChannel channel;
+
+    private IEndpointSnitch snitch;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Before
+    public void setup()
+    {
+        connectionId = OutboundConnectionIdentifier.small(LOCAL_ADDR, REMOTE_ADDR);
+        omc = new OutboundMessagingConnection(connectionId, null, Optional.empty(), new AllowAllInternodeAuthenticator());
+        channel = new EmbeddedChannel();
+        omc.setChannelWriter(ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()));
+
+        snitch = DatabaseDescriptor.getEndpointSnitch();
+    }
+
+    @After
+    public void tearDown()
+    {
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        channel.finishAndReleaseAll();
+    }
+
+    @Test
+    public void sendMessage_CreatingChannel()
+    {
+        Assert.assertEquals(0, omc.backlogSize());
+        omc.setState(CREATING_CHANNEL);
+        Assert.assertTrue(omc.sendMessage(new MessageOut<>(ECHO), 1));
+        Assert.assertEquals(1, omc.backlogSize());
+        Assert.assertEquals(1, omc.getPendingMessages().intValue());
+    }
+
+    @Test
+    public void sendMessage_HappyPath()
+    {
+        Assert.assertEquals(0, omc.backlogSize());
+        omc.setState(READY);
+        Assert.assertTrue(omc.sendMessage(new MessageOut<>(ECHO), 1));
+        Assert.assertEquals(0, omc.backlogSize());
+        Assert.assertTrue(channel.releaseOutbound());
+    }
+
+    @Test
+    public void sendMessage_Closed()
+    {
+        Assert.assertEquals(0, omc.backlogSize());
+        omc.setState(CLOSED);
+        Assert.assertFalse(omc.sendMessage(new MessageOut<>(ECHO), 1));
+        Assert.assertEquals(0, omc.backlogSize());
+        Assert.assertFalse(channel.releaseOutbound());
+    }
+
+    @Test
+    public void shouldCompressConnection_None()
+    {
+        DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.none);
+        Assert.assertFalse(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR.getAddress(), REMOTE_ADDR.getAddress()));
+    }
+
+    @Test
+    public void shouldCompressConnection_All()
+    {
+        DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.all);
+        Assert.assertTrue(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR.getAddress(), REMOTE_ADDR.getAddress()));
+    }
+
+    @Test
+    public void shouldCompressConnection_SameDc()
+    {
+        TestSnitch snitch = new TestSnitch();
+        snitch.add(LOCAL_ADDR.getAddress(), "dc1");
+        snitch.add(REMOTE_ADDR.getAddress(), "dc1");
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc);
+        Assert.assertFalse(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR.getAddress(), REMOTE_ADDR.getAddress()));
+    }
+
+    private static class TestSnitch extends AbstractEndpointSnitch
+    {
+        private Map<InetAddress, String> nodeToDc = new HashMap<>();
+
+        void add(InetAddress node, String dc)
+        {
+            nodeToDc.put(node, dc);
+        }
+
+        public String getRack(InetAddress endpoint)
+        {
+            return null;
+        }
+
+        public String getDatacenter(InetAddress endpoint)
+        {
+            return nodeToDc.get(endpoint);
+        }
+
+        public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+        {
+            return 0;
+        }
+    }
+
+    @Test
+    public void shouldCompressConnection_DifferentDc()
+    {
+        TestSnitch snitch = new TestSnitch();
+        snitch.add(LOCAL_ADDR.getAddress(), "dc1");
+        snitch.add(REMOTE_ADDR.getAddress(), "dc2");
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc);
+        Assert.assertTrue(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR.getAddress(), REMOTE_ADDR.getAddress()));
+    }
+
+    @Test
+    public void close_softClose()
+    {
+        close(true);
+    }
+
+    @Test
+    public void close_hardClose()
+    {
+        close(false);
+    }
+
+    private void close(boolean softClose)
+    {
+        int count = 32;
+        for (int i = 0; i < count; i++)
+            omc.addToBacklog(new QueuedMessage(new MessageOut<>(ECHO), i));
+        Assert.assertEquals(count, omc.backlogSize());
+        Assert.assertEquals(count, omc.getPendingMessages().intValue());
+
+        ScheduledFuture<?> connectionTimeoutFuture = new TestScheduledFuture();
+        Assert.assertFalse(connectionTimeoutFuture.isCancelled());
+        omc.setConnectionTimeoutFuture(connectionTimeoutFuture);
+        ChannelWriter channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty());
+        omc.setChannelWriter(channelWriter);
+
+        omc.close(softClose);
+        Assert.assertFalse(channel.isActive());
+        Assert.assertEquals(State.CLOSED, omc.getState());
+        Assert.assertEquals(0, omc.backlogSize());
+        Assert.assertEquals(0, omc.getPendingMessages().intValue());
+        int sentMessages = channel.outboundMessages().size();
+
+        if (softClose)
+            Assert.assertTrue(count <= sentMessages);
+        else
+            Assert.assertEquals(0, sentMessages);
+        Assert.assertTrue(connectionTimeoutFuture.isCancelled());
+        Assert.assertTrue(channelWriter.isClosed());
+    }
+
+    @Test
+    public void connect_IInternodeAuthFail()
+    {
+        IInternodeAuthenticator auth = new IInternodeAuthenticator()
+        {
+            public boolean authenticate(InetAddress remoteAddress, int remotePort)
+            {
+                return false;
+            }
+
+            public void validateConfiguration() throws ConfigurationException
+            {
+
+            }
+        };
+
+        MessageOut messageOut = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK);
+        OutboundMessagingPool pool = new OutboundMessagingPool(REMOTE_ADDR, LOCAL_ADDR, null,
+                                                               new MessagingServiceTest.MockBackPressureStrategy(null).newState(REMOTE_ADDR.getAddress()), auth);
+        omc = pool.getConnection(messageOut);
+        Assert.assertSame(State.NOT_READY, omc.getState());
+        Assert.assertFalse(omc.connect());
+    }
+
+    @Test
+    public void connect_ConnectionAlreadyStarted()
+    {
+        omc.setState(State.CREATING_CHANNEL);
+        Assert.assertFalse(omc.connect());
+        Assert.assertSame(State.CREATING_CHANNEL, omc.getState());
+    }
+
+    @Test
+    public void connect_ConnectionClosed()
+    {
+        omc.setState(State.CLOSED);
+        Assert.assertFalse(omc.connect());
+        Assert.assertSame(State.CLOSED, omc.getState());
+    }
+
+    @Test
+    public void connectionTimeout_StateIsReady()
+    {
+        omc.setState(READY);
+        ChannelFuture channelFuture = channel.newPromise();
+        Assert.assertFalse(omc.connectionTimeout(channelFuture));
+        Assert.assertEquals(READY, omc.getState());
+    }
+
+    @Test
+    public void connectionTimeout_StateIsClosed()
+    {
+        omc.setState(CLOSED);
+        ChannelFuture channelFuture = channel.newPromise();
+        Assert.assertTrue(omc.connectionTimeout(channelFuture));
+        Assert.assertEquals(CLOSED, omc.getState());
+    }
+
+    @Test
+    public void connectionTimeout_AssumeConnectionTimedOut()
+    {
+        int count = 32;
+        for (int i = 0; i < count; i++)
+            omc.addToBacklog(new QueuedMessage(new MessageOut<>(ECHO), i));
+        Assert.assertEquals(count, omc.backlogSize());
+        Assert.assertEquals(count, omc.getPendingMessages().intValue());
+
+        omc.setState(CREATING_CHANNEL);
+        ChannelFuture channelFuture = channel.newPromise();
+        Assert.assertTrue(omc.connectionTimeout(channelFuture));
+        Assert.assertEquals(NOT_READY, omc.getState());
+        Assert.assertEquals(0, omc.backlogSize());
+        Assert.assertEquals(0, omc.getPendingMessages().intValue());
+    }
+
+    @Test
+    public void connectCallback_FutureIsSuccess()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.setSuccess();
+        Assert.assertTrue(omc.connectCallback(promise));
+    }
+
+    @Test
+    public void connectCallback_Closed()
+    {
+        ChannelPromise promise = channel.newPromise();
+        omc.setState(State.CLOSED);
+        Assert.assertFalse(omc.connectCallback(promise));
+    }
+
+    @Test
+    public void connectCallback_FailCauseIsSslHandshake()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.setFailure(new SSLHandshakeException("test is only a test"));
+        Assert.assertFalse(omc.connectCallback(promise));
+        Assert.assertSame(State.NOT_READY, omc.getState());
+    }
+
+    @Test
+    public void connectCallback_FailCauseIsNPE()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.setFailure(new NullPointerException("test is only a test"));
+        Assert.assertFalse(omc.connectCallback(promise));
+        Assert.assertSame(State.NOT_READY, omc.getState());
+    }
+
+    @Test
+    public void connectCallback_FailCauseIsIOException()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.setFailure(new IOException("test is only a test"));
+        Assert.assertFalse(omc.connectCallback(promise));
+        Assert.assertSame(State.NOT_READY, omc.getState());
+    }
+
+    @Test
+    public void connectCallback_FailedAndItsClosed()
+    {
+        ChannelPromise promise = channel.newPromise();
+        promise.setFailure(new IOException("test is only a test"));
+        omc.setState(CLOSED);
+        Assert.assertFalse(omc.connectCallback(promise));
+        Assert.assertSame(State.CLOSED, omc.getState());
+    }
+
+    @Test
+    public void finishHandshake_GOOD()
+    {
+        ChannelWriter channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty());
+        HandshakeResult result = HandshakeResult.success(channelWriter, MESSAGING_VERSION);
+        ScheduledFuture<?> connectionTimeoutFuture = new TestScheduledFuture();
+        Assert.assertFalse(connectionTimeoutFuture.isCancelled());
+
+        omc.setChannelWriter(null);
+        omc.setConnectionTimeoutFuture(connectionTimeoutFuture);
+        omc.finishHandshake(result);
+        Assert.assertFalse(channelWriter.isClosed());
+        Assert.assertEquals(channelWriter, omc.getChannelWriter());
+        Assert.assertEquals(READY, omc.getState());
+        Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR.getAddress()));
+        Assert.assertNull(omc.getConnectionTimeoutFuture());
+        Assert.assertTrue(connectionTimeoutFuture.isCancelled());
+    }
+
+    @Test
+    public void finishHandshake_GOOD_ButClosed()
+    {
+        ChannelWriter channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty());
+        HandshakeResult result = HandshakeResult.success(channelWriter, MESSAGING_VERSION);
+        ScheduledFuture<?> connectionTimeoutFuture = new TestScheduledFuture();
+        Assert.assertFalse(connectionTimeoutFuture.isCancelled());
+
+        omc.setChannelWriter(null);
+        omc.setState(CLOSED);
+        omc.setConnectionTimeoutFuture(connectionTimeoutFuture);
+        omc.finishHandshake(result);
+        Assert.assertTrue(channelWriter.isClosed());
+        Assert.assertNull(omc.getChannelWriter());
+        Assert.assertEquals(CLOSED, omc.getState());
+        Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR.getAddress()));
+        Assert.assertNull(omc.getConnectionTimeoutFuture());
+        Assert.assertTrue(connectionTimeoutFuture.isCancelled());
+    }
+
+    @Test
+    public void finishHandshake_DISCONNECT()
+    {
+        int count = 32;
+        for (int i = 0; i < count; i++)
+            omc.addToBacklog(new QueuedMessage(new MessageOut<>(ECHO), i));
+        Assert.assertEquals(count, omc.backlogSize());
+
+        HandshakeResult result = HandshakeResult.disconnect(MESSAGING_VERSION);
+        omc.finishHandshake(result);
+        Assert.assertNotNull(omc.getChannelWriter());
+        Assert.assertEquals(CREATING_CHANNEL, omc.getState());
+        Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR.getAddress()));
+        Assert.assertEquals(count, omc.backlogSize());
+    }
+
+    @Test
+    public void finishHandshake_CONNECT_FAILURE()
+    {
+        int count = 32;
+        for (int i = 0; i < count; i++)
+            omc.addToBacklog(new QueuedMessage(new MessageOut<>(ECHO), i));
+        Assert.assertEquals(count, omc.backlogSize());
+
+        HandshakeResult result = HandshakeResult.failed();
+        omc.finishHandshake(result);
+        Assert.assertEquals(NOT_READY, omc.getState());
+        Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR.getAddress()));
+        Assert.assertEquals(0, omc.backlogSize());
+    }
+
+    @Test
+    public void setStateIfNotClosed_AlreadyClosed()
+    {
+        AtomicReference<State> state = new AtomicReference<>(CLOSED);
+        OutboundMessagingConnection.setStateIfNotClosed(state, NOT_READY);
+        Assert.assertEquals(CLOSED, state.get());
+    }
+
+    @Test
+    public void setStateIfNotClosed_NotClosed()
+    {
+        AtomicReference<State> state = new AtomicReference<>(READY);
+        OutboundMessagingConnection.setStateIfNotClosed(state, NOT_READY);
+        Assert.assertEquals(NOT_READY, state.get());
+    }
+
+    @Test
+    public void reconnectWithNewIp_HappyPath()
+    {
+        ChannelWriter channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty());
+        omc.setChannelWriter(channelWriter);
+        omc.setState(READY);
+        OutboundConnectionIdentifier originalId = omc.getConnectionId();
+        omc.reconnectWithNewIp(RECONNECT_ADDR);
+        Assert.assertFalse(omc.getConnectionId().equals(originalId));
+        Assert.assertTrue(channelWriter.isClosed());
+        Assert.assertNotSame(CLOSED, omc.getState());
+    }
+
+    @Test
+    public void reconnectWithNewIp_Closed()
+    {
+        omc.setState(CLOSED);
+        OutboundConnectionIdentifier originalId = omc.getConnectionId();
+        omc.reconnectWithNewIp(RECONNECT_ADDR);
+        Assert.assertSame(omc.getConnectionId(), originalId);
+        Assert.assertSame(CLOSED, omc.getState());
+    }
+
+    @Test
+    public void reconnectWithNewIp_UnsedConnection()
+    {
+        omc.setState(NOT_READY);
+        OutboundConnectionIdentifier originalId = omc.getConnectionId();
+        omc.reconnectWithNewIp(RECONNECT_ADDR);
+        Assert.assertNotSame(omc.getConnectionId(), originalId);
+        Assert.assertSame(NOT_READY, omc.getState());
+    }
+
+    private static class TestScheduledFuture implements ScheduledFuture<Object>
+    {
+        private boolean cancelled = false;
+
+        public long getDelay(TimeUnit unit)
+        {
+            return 0;
+        }
+
+        public int compareTo(Delayed o)
+        {
+            return 0;
+        }
+
+        public boolean cancel(boolean mayInterruptIfRunning)
+        {
+            cancelled = true;
+            return false;
+        }
+
+        public boolean isCancelled()
+        {
+            return cancelled;
+        }
+
+        public boolean isDone()
+        {
+            return false;
+        }
+
+        public Object get() throws InterruptedException, ExecutionException
+        {
+            return null;
+        }
+
+        public Object get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+        {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java b/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java
new file mode 100644
index 0000000..655cd15
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.async;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.WriteResponse;
+import org.apache.cassandra.gms.GossipDigestSyn;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.BackPressureState;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType;
+
+public class OutboundMessagingPoolTest
+{
+    private static final InetSocketAddress LOCAL_ADDR = new InetSocketAddress("127.0.0.1", 9476);
+    private static final InetSocketAddress REMOTE_ADDR = new InetSocketAddress("127.0.0.2", 9476);
+    private static final InetSocketAddress RECONNECT_ADDR = new InetSocketAddress("127.0.0.3", 9476);
+    private static final List<ConnectionType> INTERNODE_MESSAGING_CONN_TYPES = new ArrayList<ConnectionType>()
+            {{ add(ConnectionType.GOSSIP); add(ConnectionType.LARGE_MESSAGE); add(ConnectionType.SMALL_MESSAGE); }};
+
+    private OutboundMessagingPool pool;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Before
+    public void setup()
+    {
+        BackPressureState backPressureState = DatabaseDescriptor.getBackPressureStrategy().newState(REMOTE_ADDR.getAddress());
+        pool = new OutboundMessagingPool(REMOTE_ADDR, LOCAL_ADDR, null, backPressureState, new AllowAllInternodeAuthenticator());
+    }
+
+    @After
+    public void tearDown()
+    {
+        if (pool != null)
+            pool.close(false);
+    }
+
+    @Test
+    public void getConnection_Gossip()
+    {
+        GossipDigestSyn syn = new GossipDigestSyn("cluster", "partitioner", new ArrayList<>(0));
+        MessageOut<GossipDigestSyn> message = new MessageOut<>(MessagingService.Verb.GOSSIP_DIGEST_SYN,
+                                                                              syn, GossipDigestSyn.serializer);
+        Assert.assertEquals(ConnectionType.GOSSIP, pool.getConnection(message).getConnectionId().type());
+    }
+
+    @Test
+    public void getConnection_SmallMessage()
+    {
+        MessageOut message = WriteResponse.createMessage();
+        Assert.assertEquals(ConnectionType.SMALL_MESSAGE, pool.getConnection(message).getConnectionId().type());
+    }
+
+    @Test
+    public void getConnection_LargeMessage()
+    {
+        // just need a serializer to report a size, as fake as it may be
+        IVersionedSerializer<Object> serializer = new IVersionedSerializer<Object>()
+        {
+            public void serialize(Object o, DataOutputPlus out, int version)
+            {
+
+            }
+
+            public Object deserialize(DataInputPlus in, int version)
+            {
+                return null;
+            }
+
+            public long serializedSize(Object o, int version)
+            {
+                return OutboundMessagingPool.LARGE_MESSAGE_THRESHOLD + 1;
+            }
+        };
+        MessageOut message = new MessageOut<>(MessagingService.Verb.UNUSED_5, "payload", serializer);
+        Assert.assertEquals(ConnectionType.LARGE_MESSAGE, pool.getConnection(message).getConnectionId().type());
+    }
+
+    @Test
+    public void close()
+    {
+        for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES)
+            Assert.assertNotSame(OutboundMessagingConnection.State.CLOSED, pool.getConnection(type).getState());
+        pool.close(false);
+        for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES)
+            Assert.assertEquals(OutboundMessagingConnection.State.CLOSED, pool.getConnection(type).getState());
+    }
+
+    @Test
+    public void reconnectWithNewIp()
+    {
+        for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES)
+        {
+            Assert.assertEquals(REMOTE_ADDR, pool.getPreferredRemoteAddr());
+            Assert.assertEquals(REMOTE_ADDR, pool.getConnection(type).getConnectionId().connectionAddress());
+        }
+
+        pool.reconnectWithNewIp(RECONNECT_ADDR);
+
+        for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES)
+        {
+            Assert.assertEquals(RECONNECT_ADDR, pool.getPreferredRemoteAddr());
+            Assert.assertEquals(RECONNECT_ADDR, pool.getConnection(type).getConnectionId().connectionAddress());
+        }
+    }
+
+    @Test
+    public void timeoutCounter()
+    {
+        long originalValue = pool.getTimeouts();
+        pool.incrementTimeout();
+        Assert.assertEquals(originalValue + 1, pool.getTimeouts());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/net/async/TestAuthenticator.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/TestAuthenticator.java b/test/unit/org/apache/cassandra/net/async/TestAuthenticator.java
new file mode 100644
index 0000000..3107f2a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/async/TestAuthenticator.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.net.async;
+
+import java.net.InetAddress;
+
+import org.apache.cassandra.auth.IInternodeAuthenticator;
+import org.apache.cassandra.exceptions.ConfigurationException;
+
+class TestAuthenticator implements IInternodeAuthenticator
+{
+    private final boolean authAll;
+
+    TestAuthenticator(boolean authAll)
+    {
+        this.authAll = authAll;
+    }
+
+    public boolean authenticate(InetAddress remoteAddress, int remotePort)
+    {
+        return authAll;
+    }
+
+    public void validateConfiguration() throws ConfigurationException
+    {   }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
index 5dc8112..2ef9446 100644
--- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
+++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
@@ -30,7 +30,9 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
 
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Murmur3Partitioner;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/security/SSLFactoryTest.java b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
index b3510bd..61933a5 100644
--- a/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
+++ b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
@@ -18,23 +18,62 @@
 */
 package org.apache.cassandra.security;
 
-import static org.junit.Assert.assertArrayEquals;
-
 import java.io.IOException;
 import java.net.InetAddress;
-
+import java.security.cert.CertificateException;
+import java.util.Arrays;
 import javax.net.ssl.SSLServerSocket;
+import javax.net.ssl.TrustManagerFactory;
 
 import com.google.common.base.Predicates;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.util.SelfSignedCertificate;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
-import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
 
 public class SSLFactoryTest
 {
+    private static final Logger logger = LoggerFactory.getLogger(SSLFactoryTest.class);
+
+    static final SelfSignedCertificate ssc;
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+        try
+        {
+            ssc = new SelfSignedCertificate();
+        }
+        catch (CertificateException e)
+        {
+            throw new RuntimeException("fialed to create test certs");
+        }
+    }
+
+    private ServerEncryptionOptions encryptionOptions;
+
+    @Before
+    public void setup()
+    {
+        encryptionOptions = new ServerEncryptionOptions();
+        encryptionOptions.truststore = "test/conf/cassandra_ssl_test.truststore";
+        encryptionOptions.truststore_password = "cassandra";
+        encryptionOptions.require_client_auth = false;
+        encryptionOptions.cipher_suites = new String[] {"TLS_RSA_WITH_AES_128_CBC_SHA"};
+
+        SSLFactory.checkedExpiry = false;
+    }
 
     @Test
     public void testFilterCipherSuites()
@@ -48,28 +87,79 @@ public class SSLFactoryTest
     }
 
     @Test
-    public void testServerSocketCiphers() throws IOException
+    public void getSslContext_OpenSSL() throws IOException
     {
-        ServerEncryptionOptions options = new EncryptionOptions.ServerEncryptionOptions();
-        options.keystore = "test/conf/keystore.jks";
-        options.keystore_password = "cassandra";
-        options.truststore = options.keystore;
-        options.truststore_password = options.keystore_password;
-        options.cipher_suites = new String[] {
-            "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA",
-            "TLS_DHE_RSA_WITH_AES_128_CBC_SHA", "TLS_DHE_RSA_WITH_AES_256_CBC_SHA",
-            "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA", "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA"
-        };
-
-        // enabled ciphers must be a subset of configured ciphers with identical order
-        try (SSLServerSocket socket = SSLFactory.getServerSocket(options, InetAddress.getLocalHost(), 55123))
+        // only try this test if OpenSsl is available
+        if (!OpenSsl.isAvailable())
         {
-            String[] enabled = socket.getEnabledCipherSuites();
-            String[] wanted = Iterables.toArray(Iterables.filter(Lists.newArrayList(options.cipher_suites),
-                                                                 Predicates.in(Lists.newArrayList(enabled))),
-                                                String.class);
-            assertArrayEquals(wanted, enabled);
+            logger.warn("OpenSSL not available in this application, so not testing the netty-openssl code paths");
+            return;
         }
+
+        EncryptionOptions options = addKeystoreOptions(encryptionOptions);
+        SslContext sslContext = SSLFactory.getSslContext(options, true, true, true);
+        Assert.assertNotNull(sslContext);
+    }
+
+    @Test
+    public void getSslContext_JdkSsl() throws IOException
+    {
+        EncryptionOptions options = addKeystoreOptions(encryptionOptions);
+        SslContext sslContext = SSLFactory.getSslContext(options, true, true, false);
+        Assert.assertNotNull(sslContext);
+        Assert.assertEquals(Arrays.asList(encryptionOptions.cipher_suites), sslContext.cipherSuites());
+    }
+
+    private EncryptionOptions addKeystoreOptions(EncryptionOptions options)
+    {
+        options.keystore = "test/conf/cassandra_ssl_test.keystore";
+        options.keystore_password = "cassandra";
+        return options;
+    }
+
+    @Test (expected = IOException.class)
+    public void buildTrustManagerFactory_NoFile() throws IOException
+    {
+        encryptionOptions.truststore = "/this/is/probably/not/a/file/on/your/test/machine";
+        SSLFactory.buildTrustManagerFactory(encryptionOptions);
+    }
+
+    @Test (expected = IOException.class)
+    public void buildTrustManagerFactory_BadPassword() throws IOException
+    {
+        encryptionOptions.truststore_password = "HomeOfBadPasswords";
+        SSLFactory.buildTrustManagerFactory(encryptionOptions);
+    }
+
+    @Test
+    public void buildTrustManagerFactory_HappyPath() throws IOException
+    {
+        TrustManagerFactory trustManagerFactory = SSLFactory.buildTrustManagerFactory(encryptionOptions);
+        Assert.assertNotNull(trustManagerFactory);
+    }
+
+    @Test (expected = IOException.class)
+    public void buildKeyManagerFactory_NoFile() throws IOException
+    {
+        EncryptionOptions options = addKeystoreOptions(encryptionOptions);
+        options.keystore = "/this/is/probably/not/a/file/on/your/test/machine";
+        SSLFactory.buildKeyManagerFactory(options);
     }
 
+    @Test (expected = IOException.class)
+    public void buildKeyManagerFactory_BadPassword() throws IOException
+    {
+        EncryptionOptions options = addKeystoreOptions(encryptionOptions);
+        encryptionOptions.keystore_password = "HomeOfBadPasswords";
+        SSLFactory.buildKeyManagerFactory(options);
+    }
+
+    @Test
+    public void buildKeyManagerFactory_HappyPath() throws IOException
+    {
+        Assert.assertFalse(SSLFactory.checkedExpiry);
+        EncryptionOptions options = addKeystoreOptions(encryptionOptions);
+        SSLFactory.buildKeyManagerFactory(options);
+        Assert.assertTrue(SSLFactory.checkedExpiry);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/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 aa9e666..6a5002e 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -26,6 +26,7 @@ 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.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.slf4j.Logger;
@@ -61,6 +62,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
+
+// TODO:JEB intentionally breaking this with CASSANDRA-8457 until CASSANDRA-12229
+@Ignore
 public class StreamingTransferTest
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamingTransferTest.class);
@@ -277,19 +281,20 @@ public class StreamingTransferTest
      */
     private void verifyConnectionsAreClosed() throws InterruptedException
     {
+        // TODO:JEB intentionally breaking this with CASSANDRA-8457 until CASSANDRA-12229
         //after stream session is finished, message handlers may take several milliseconds to be closed
-        outer:
-        for (int i = 0; i <= 100; i++)
-        {
-            for (MessagingService.SocketThread socketThread : MessagingService.instance().getSocketThreads())
-                if (!socketThread.connections.isEmpty())
-                {
-                    Thread.sleep(100);
-                    continue outer;
-                }
-            return;
-        }
-        fail("Streaming connections remain registered in MessagingService");
+//        outer:
+//        for (int i = 0; i <= 100; i++)
+//        {
+//            for (MessagingService.SocketThread socketThread : MessagingService.instance().getSocketThreads())
+//                if (!socketThread.connections.isEmpty())
+//                {
+//                    Thread.sleep(100);
+//                    continue outer;
+//                }
+//            return;
+//        }
+//        fail("Streaming connections remain registered in MessagingService");
     }
 
     private Collection<StreamSession.SSTableStreamingSections> makeStreamingDetails(List<Range<Token>> ranges, Refs<SSTableReader> sstables)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/356dc3c2/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java b/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java
index b10d70b..8877fe9 100644
--- a/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java
+++ b/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java
@@ -17,60 +17,25 @@
  */
 package org.apache.cassandra.utils;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.utils.CoalescingStrategies.Clock;
-import org.apache.cassandra.utils.CoalescingStrategies.Coalescable;
-import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
-import org.apache.cassandra.utils.CoalescingStrategies.Parker;
-import org.junit.BeforeClass;
-import org.junit.Before;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Queue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.LockSupport;
-
-import static org.junit.Assert.*;
+import org.apache.cassandra.utils.CoalescingStrategies.Coalescable;
+import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
+import org.apache.cassandra.utils.CoalescingStrategies.FixedCoalescingStrategy;
+import org.apache.cassandra.utils.CoalescingStrategies.MovingAverageCoalescingStrategy;
+import org.apache.cassandra.utils.CoalescingStrategies.TimeHorizonMovingAverageCoalescingStrategy;
 
 public class CoalescingStrategiesTest
 {
-
-    static final ExecutorService ex = Executors.newSingleThreadExecutor();
-
     private static final Logger logger = LoggerFactory.getLogger(CoalescingStrategiesTest.class);
-
-    static class MockParker implements Parker
-    {
-        Queue<Long> parks = new ArrayDeque<Long>();
-        Semaphore permits = new Semaphore(0);
-
-        Semaphore parked = new Semaphore(0);
-
-        public void park(long nanos)
-        {
-            parks.offer(nanos);
-            parked.release();
-            try
-            {
-                permits.acquire();
-            }
-            catch (InterruptedException e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-    }
+    private static final int WINDOW_IN_MICROS = 200;
+    private static final long WINDOW_IN_NANOS = TimeUnit.MICROSECONDS.toNanos(WINDOW_IN_MICROS);
+    private static final String DISPLAY_NAME = "Stupendopotamus";
 
     static class SimpleCoalescable implements Coalescable
     {
@@ -87,399 +52,73 @@ public class CoalescingStrategiesTest
         }
     }
 
-
     static long toNanos(long micros)
     {
         return TimeUnit.MICROSECONDS.toNanos(micros);
     }
 
-    MockParker parker;
-
-    BlockingQueue<SimpleCoalescable> input;
-    List<SimpleCoalescable> output;
-
-    CoalescingStrategy cs;
-
-    Semaphore queueParked = new Semaphore(0);
-    Semaphore queueRelease = new Semaphore(0);
-
-    @BeforeClass
-    public static void initDD()
-    {
-        DatabaseDescriptor.daemonInitialization();
-    }
-
-    @SuppressWarnings({ "serial" })
-    @Before
-    public void setUp() throws Exception
-    {
-        cs = null;
-        CoalescingStrategies.CLOCK = new Clock()
-        {
-            @Override
-            public long nanoTime()
-            {
-                return 0;
-            }
-        };
-
-        parker = new MockParker();
-        input = new LinkedBlockingQueue<SimpleCoalescable>()
-                {
-            @Override
-            public SimpleCoalescable take() throws InterruptedException
-            {
-                queueParked.release();
-                queueRelease.acquire();
-                return super.take();
-            }
-        };
-        output = new ArrayList<>(128);
-
-        clear();
-    }
-
-    CoalescingStrategy newStrategy(String name, int window)
-    {
-        return CoalescingStrategies.newCoalescingStrategy(name, window, parker, logger, "Stupendopotamus");
-    }
-
-    void add(long whenMicros)
-    {
-        input.offer(new SimpleCoalescable(toNanos(whenMicros)));
-    }
-
-    void clear()
-    {
-        output.clear();
-        input.clear();
-        parker.parks.clear();
-        parker.parked.drainPermits();
-        parker.permits.drainPermits();
-        queueParked.drainPermits();
-        queueRelease.drainPermits();
-    }
-
-    void release() throws Exception
-    {
-        queueRelease.release();
-        parker.permits.release();
-        fut.get();
-    }
-
-    Future<?> fut;
-    void runBlocker(Semaphore waitFor) throws Exception
-    {
-        fut = ex.submit(new Runnable()
-        {
-            @Override
-            public void run()
-            {
-                try
-                {
-                    cs.coalesce(input, output, 128);
-                }
-                catch (Exception ex)
-                {
-                    ex.printStackTrace();
-                    throw new RuntimeException(ex);
-                }
-            }
-        });
-        waitFor.acquire();
-    }
-
     @Test
-    public void testFixedCoalescingStrategy() throws Exception
+    public void testFixedCoalescingStrategy()
     {
-        cs = newStrategy("FIXED", 200);
-
-        //Test that when a stream of messages continues arriving it keeps sending until all are drained
-        //It does this because it is already awake and sending messages
-        add(42);
-        add(42);
-        cs.coalesce(input, output, 128);
-        assertEquals( 2, output.size());
-        assertNull(parker.parks.poll());
-
-        clear();
-
-        runBlocker(queueParked);
-        add(42);
-        add(42);
-        add(42);
-        release();
-        assertEquals( 3, output.size());
-        assertEquals(toNanos(200), parker.parks.poll().longValue());
-
+        CoalescingStrategy cs = new FixedCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME);
+        Assert.assertEquals(WINDOW_IN_NANOS, cs.currentCoalescingTimeNanos());
     }
 
     @Test
-    public void testFixedCoalescingStrategyEnough() throws Exception
+    public void testMovingAverageCoalescingStrategy_DoCoalesce()
     {
-        int oldValue = DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages();
-        DatabaseDescriptor.setOtcCoalescingEnoughCoalescedMessages(1);
-        try {
-            cs = newStrategy("FIXED", 200);
-
-            //Test that when a stream of messages continues arriving it keeps sending until all are drained
-            //It does this because it is already awake and sending messages
-            add(42);
-            add(42);
-            cs.coalesce(input, output, 128);
-            assertEquals(2, output.size());
-            assertNull(parker.parks.poll());
-
-            clear();
-
-            runBlocker(queueParked);
-            add(42);
-            add(42);
-            add(42);
-            release();
-            assertEquals(3, output.size());
-            assertNull(parker.parks.poll());
-        }
-        finally {
-            DatabaseDescriptor.setOtcCoalescingEnoughCoalescedMessages(oldValue);
-        }
+        CoalescingStrategy cs = new MovingAverageCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME);
 
+        for (int i = 0; i < MovingAverageCoalescingStrategy.SAMPLE_SIZE; i++)
+            cs.newArrival(new SimpleCoalescable(toNanos(i)));
+        Assert.assertTrue(0 < cs.currentCoalescingTimeNanos());
     }
 
     @Test
-    public void testDisabledCoalescingStrateg() throws Exception
+    public void testMovingAverageCoalescingStrategy_DoNotCoalesce()
     {
-        cs = newStrategy("DISABLED", 200);
-
-        add(42);
-        add(42);
-        cs.coalesce(input, output, 128);
-        assertEquals( 2, output.size());
-        assertNull(parker.parks.poll());
-
-        clear();
+        CoalescingStrategy cs = new MovingAverageCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME);
 
-        runBlocker(queueParked);
-        add(42);
-        add(42);
-        release();
-        assertEquals( 2, output.size());
-        assertNull(parker.parks.poll());
+        for (int i = 0; i < MovingAverageCoalescingStrategy.SAMPLE_SIZE; i++)
+            cs.newArrival(new SimpleCoalescable(toNanos(WINDOW_IN_MICROS + i) * i));
+        Assert.assertTrue(0 >= cs.currentCoalescingTimeNanos());
     }
 
     @Test
-    public void parkLoop() throws Exception
-   {
-        final Thread current = Thread.currentThread();
-        final Semaphore helperReady = new Semaphore(0);
-        final Semaphore helperGo = new Semaphore(0);
-
-        new Thread()
-        {
-            @Override
-            public void run()
-            {
-                try
-                {
-                    helperReady.release();
-                    helperGo.acquire();
-                    Thread.sleep(50);
-                    LockSupport.unpark(current);
-                }
-                catch (Exception e)
-                {
-                    e.printStackTrace();
-                    logger.error("Error", e);
-                    System.exit(-1);
-                }
-            }
-        }.start();
-
-        long start = System.nanoTime();
-        helperGo.release();
-
-        long parkNanos = TimeUnit.MILLISECONDS.toNanos(500);
-
-        CoalescingStrategies.parkLoop(parkNanos);
-        long delta = System.nanoTime() - start;
+    public void testTimeHorizonStrategy_DoCoalesce()
+    {
+        long initialEpoch = 0;
+        CoalescingStrategy cs = new TimeHorizonMovingAverageCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME, initialEpoch);
 
-        assertTrue (delta >= (parkNanos - (parkNanos / 16)));
+        for (int i = 0; i < 10_000; i++)
+            cs.newArrival(new SimpleCoalescable(toNanos(i)));
+        Assert.assertTrue(0 < cs.currentCoalescingTimeNanos());
     }
 
     @Test
-    public void testMovingAverageCoalescingStrategy() throws Exception
+    public void testTimeHorizonStrategy_DoNotCoalesce()
     {
-        cs = newStrategy("org.apache.cassandra.utils.CoalescingStrategies$MovingAverageCoalescingStrategy", 200);
-
-
-        //Test that things can be pulled out of the queue if it is non-empty
-        add(201);
-        add(401);
-        cs.coalesce(input, output, 128);
-        assertEquals( 2, output.size());
-        assertNull(parker.parks.poll());
-
-        //Test that blocking on the queue results in everything drained
-        clear();
-
-        runBlocker(queueParked);
-        add(601);
-        add(801);
-        release();
-        assertEquals( 2, output.size());
-        assertNull(parker.parks.poll());
-
-        clear();
-
-        //Test that out of order samples still flow
-        runBlocker(queueParked);
-        add(0);
-        release();
-        assertEquals( 1, output.size());
-        assertNull(parker.parks.poll());
-
-        clear();
-
-        add(0);
-        cs.coalesce(input, output, 128);
-        assertEquals( 1, output.size());
-        assertNull(parker.parks.poll());
-
-        clear();
-
-        //Test that too high an average doesn't coalesce
-        for (long ii = 0; ii < 128; ii++)
-            add(ii * 1000);
-        cs.coalesce(input, output, 128);
-        assertEquals(output.size(), 128);
-        assertTrue(parker.parks.isEmpty());
-
-        clear();
-
-        runBlocker(queueParked);
-        add(129 * 1000);
-        release();
-        assertTrue(parker.parks.isEmpty());
+        long initialEpoch = 0;
+        CoalescingStrategy cs = new TimeHorizonMovingAverageCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME, initialEpoch);
 
-        clear();
-
-        //Test that a low enough average coalesces
-        cs = newStrategy("MOVINGAVERAGE", 200);
-        for (long ii = 0; ii < 128; ii++)
-            add(ii * 99);
-        cs.coalesce(input, output, 128);
-        assertEquals(output.size(), 128);
-        assertTrue(parker.parks.isEmpty());
-
-        clear();
-
-        runBlocker(queueParked);
-        add(128 * 99);
-        add(129 * 99);
-        release();
-        assertEquals(2, output.size());
-        assertEquals(toNanos(198), parker.parks.poll().longValue());
+        for (int i = 0; i < 1_000_000; i++)
+            cs.newArrival(new SimpleCoalescable(toNanos(WINDOW_IN_MICROS + i) * i));
+        Assert.assertTrue(0 >= cs.currentCoalescingTimeNanos());
     }
 
     @Test
-    public void testTimeHorizonStrategy() throws Exception
+    public void determineCoalescingTime_LargeAverageGap()
     {
-        cs = newStrategy("TIMEHORIZON", 200);
-
-        //Test that things can be pulled out of the queue if it is non-empty
-        add(201);
-        add(401);
-        cs.coalesce(input, output, 128);
-        assertEquals( 2, output.size());
-        assertNull(parker.parks.poll());
-
-        //Test that blocking on the queue results in everything drained
-        clear();
-
-        runBlocker(queueParked);
-        add(601);
-        add(801);
-        release();
-        assertEquals( 2, output.size());
-        assertNull(parker.parks.poll());
-
-        clear();
-
-        //Test that out of order samples still flow
-        runBlocker(queueParked);
-        add(0);
-        release();
-        assertEquals( 1, output.size());
-        assertNull(parker.parks.poll());
-
-        clear();
-
-        add(0);
-        cs.coalesce(input, output, 128);
-        assertEquals( 1, output.size());
-        assertNull(parker.parks.poll());
-
-        clear();
-
-        //Test that too high an average doesn't coalesce
-        for (long ii = 0; ii < 128; ii++)
-            add(ii * 1000);
-        cs.coalesce(input, output, 128);
-        assertEquals(output.size(), 128);
-        assertTrue(parker.parks.isEmpty());
-
-        clear();
-
-        runBlocker(queueParked);
-        add(129 * 1000);
-        release();
-        assertTrue(parker.parks.isEmpty());
-
-        clear();
-
-        //Test that a low enough average coalesces
-        cs = newStrategy("TIMEHORIZON", 200);
-        primeTimeHorizonAverage(99);
-
-        clear();
-
-        runBlocker(queueParked);
-        add(100000 * 99);
-        queueRelease.release();
-        parker.parked.acquire();
-        add(100001 * 99);
-        parker.permits.release();
-        fut.get();
-        assertEquals(2, output.size());
-        assertEquals(toNanos(198), parker.parks.poll().longValue());
-
-        clear();
-
-        //Test far future
-        add(Integer.MAX_VALUE);
-        cs.coalesce(input, output, 128);
-        assertEquals(1, output.size());
-        assertTrue(parker.parks.isEmpty());
-
-        clear();
-
-        //Distant past
-        add(0);
-        cs.coalesce(input, output, 128);
-        assertEquals(1, output.size());
-        assertTrue(parker.parks.isEmpty());
+        Assert.assertTrue(0 >= CoalescingStrategies.determineCoalescingTime(WINDOW_IN_NANOS * 2, WINDOW_IN_NANOS));
+        Assert.assertTrue(0 >= CoalescingStrategies.determineCoalescingTime(Integer.MAX_VALUE, WINDOW_IN_NANOS));
     }
 
-    void primeTimeHorizonAverage(long micros) throws Exception
+    @Test
+    public void determineCoalescingTime_SmallAvgGap()
     {
-        for (long ii = 0; ii < 100000; ii++)
-        {
-            add(ii * micros);
-            if (ii % 128 == 0)
-            {
-                cs.coalesce(input, output, 128);
-                output.clear();
-            }
-        }
+        Assert.assertTrue(WINDOW_IN_NANOS >= CoalescingStrategies.determineCoalescingTime(WINDOW_IN_NANOS / 2, WINDOW_IN_NANOS));
+        Assert.assertTrue(WINDOW_IN_NANOS >= CoalescingStrategies.determineCoalescingTime(WINDOW_IN_NANOS - 1, WINDOW_IN_NANOS));
+        Assert.assertTrue(WINDOW_IN_NANOS >= CoalescingStrategies.determineCoalescingTime(1, WINDOW_IN_NANOS));
+        Assert.assertEquals(WINDOW_IN_NANOS, CoalescingStrategies.determineCoalescingTime(0, WINDOW_IN_NANOS));
     }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org