You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by aw...@apache.org on 2018/01/25 20:11:58 UTC

[01/19] cassandra git commit: Allow storage port to be configurable per node

Repository: cassandra
Updated Branches:
  refs/heads/trunk 4de7a65ed -> 59b5b6bef


http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 3ef7bbb..3884f5a 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -23,7 +23,6 @@ import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.net.InetAddress;
 import java.util.*;
 
 import com.google.common.collect.HashMultimap;
@@ -35,6 +34,7 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.Schema;
@@ -192,12 +192,12 @@ public class StorageServiceServerTest
         metadata.clearUnsafe();
 
         // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.2"));
 
         // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddressAndPort.getByName("127.0.0.5"));
 
         Map<String, String> configOptions = new HashMap<>();
         configOptions.put("DC1", "1");
@@ -209,22 +209,22 @@ public class StorageServiceServerTest
         Schema.instance.load(meta);
 
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name,
-                                                                                                            InetAddress.getByName("127.0.0.1"));
+                                                                                                            InetAddressAndPort.getByName("127.0.0.1"));
         assertEquals(2, primaryRanges.size());
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("D"), new StringToken("A"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("D"))));
 
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.2"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.2"));
         assertEquals(2, primaryRanges.size());
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("B"), new StringToken("C"))));
 
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.4"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.4"));
         assertEquals(2, primaryRanges.size());
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("D"), new StringToken("A"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B"))));
 
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.5"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.5"));
         assertEquals(2, primaryRanges.size());
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("B"), new StringToken("C"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("D"))));
@@ -236,11 +236,11 @@ public class StorageServiceServerTest
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
         // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.2"));
         // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddressAndPort.getByName("127.0.0.5"));
 
         Map<String, String> configOptions = new HashMap<>();
         configOptions.put("DC1", "1");
@@ -251,19 +251,19 @@ public class StorageServiceServerTest
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, configOptions));
         Schema.instance.load(meta);
 
-        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
+        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.1"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("D"), new StringToken("A")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.2"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.2"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("B"), new StringToken("C")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.4"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.4"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.5"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.5"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("D")));
     }
@@ -274,11 +274,11 @@ public class StorageServiceServerTest
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
         // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.2"));
         // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddressAndPort.getByName("127.0.0.5"));
 
         Map<String, String> configOptions = new HashMap<>();
         configOptions.put("DC2", "2");
@@ -289,19 +289,19 @@ public class StorageServiceServerTest
         Schema.instance.load(meta);
 
         // endpoints in DC1 should not have primary range
-        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
+        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.1"));
         assert primaryRanges.isEmpty();
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.2"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.2"));
         assert primaryRanges.isEmpty();
 
         // endpoints in DC2 should have primary ranges which also cover DC1
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.4"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.4"));
         assert primaryRanges.size() == 2;
         assert primaryRanges.contains(new Range<Token>(new StringToken("D"), new StringToken("A")));
         assert primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.5"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.5"));
         assert primaryRanges.size() == 2;
         assert primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("D")));
         assert primaryRanges.contains(new Range<Token>(new StringToken("B"), new StringToken("C")));
@@ -313,11 +313,11 @@ public class StorageServiceServerTest
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
         // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.2"));
         // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddressAndPort.getByName("127.0.0.5"));
 
         Map<String, String> configOptions = new HashMap<>();
         configOptions.put("DC2", "2");
@@ -328,20 +328,20 @@ public class StorageServiceServerTest
         Schema.instance.load(meta);
 
         // endpoints in DC1 should not have primary range
-        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.1"));
+        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.1"));
         assertTrue(primaryRanges.isEmpty());
 
         primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name,
-                                                                                   InetAddress.getByName("127.0.0.2"));
+                                                                                   InetAddressAndPort.getByName("127.0.0.2"));
         assertTrue(primaryRanges.isEmpty());
 
         // endpoints in DC2 should have primary ranges which also cover DC1
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.4"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.4"));
         assertTrue(primaryRanges.size() == 2);
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("D"), new StringToken("A"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B"))));
 
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.5"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.5"));
         assertTrue(primaryRanges.size() == 2);
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("D"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("B"), new StringToken("C"))));
@@ -353,22 +353,22 @@ public class StorageServiceServerTest
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
         // DC1
-        Multimap<InetAddress, Token> dc1 = HashMultimap.create();
-        dc1.put(InetAddress.getByName("127.0.0.1"), new StringToken("A"));
-        dc1.put(InetAddress.getByName("127.0.0.1"), new StringToken("E"));
-        dc1.put(InetAddress.getByName("127.0.0.1"), new StringToken("H"));
-        dc1.put(InetAddress.getByName("127.0.0.2"), new StringToken("C"));
-        dc1.put(InetAddress.getByName("127.0.0.2"), new StringToken("I"));
-        dc1.put(InetAddress.getByName("127.0.0.2"), new StringToken("J"));
+        Multimap<InetAddressAndPort, Token> dc1 = HashMultimap.create();
+        dc1.put(InetAddressAndPort.getByName("127.0.0.1"), new StringToken("A"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.1"), new StringToken("E"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.1"), new StringToken("H"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.2"), new StringToken("C"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.2"), new StringToken("I"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.2"), new StringToken("J"));
         metadata.updateNormalTokens(dc1);
         // DC2
-        Multimap<InetAddress, Token> dc2 = HashMultimap.create();
-        dc2.put(InetAddress.getByName("127.0.0.4"), new StringToken("B"));
-        dc2.put(InetAddress.getByName("127.0.0.4"), new StringToken("G"));
-        dc2.put(InetAddress.getByName("127.0.0.4"), new StringToken("L"));
-        dc2.put(InetAddress.getByName("127.0.0.5"), new StringToken("D"));
-        dc2.put(InetAddress.getByName("127.0.0.5"), new StringToken("F"));
-        dc2.put(InetAddress.getByName("127.0.0.5"), new StringToken("K"));
+        Multimap<InetAddressAndPort, Token> dc2 = HashMultimap.create();
+        dc2.put(InetAddressAndPort.getByName("127.0.0.4"), new StringToken("B"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.4"), new StringToken("G"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.4"), new StringToken("L"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.5"), new StringToken("D"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.5"), new StringToken("F"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.5"), new StringToken("K"));
         metadata.updateNormalTokens(dc2);
 
         Map<String, String> configOptions = new HashMap<>();
@@ -380,14 +380,14 @@ public class StorageServiceServerTest
         Schema.instance.load(meta);
 
         // endpoints in DC1 should not have primary range
-        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
+        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.1"));
         assert primaryRanges.isEmpty();
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.2"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.2"));
         assert primaryRanges.isEmpty();
 
         // endpoints in DC2 should have primary ranges which also cover DC1
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.4"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.4"));
         assert primaryRanges.size() == 4;
         assert primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B")));
         assert primaryRanges.contains(new Range<Token>(new StringToken("F"), new StringToken("G")));
@@ -396,7 +396,7 @@ public class StorageServiceServerTest
         // the node covers range (L, A]
         assert primaryRanges.contains(new Range<Token>(new StringToken("L"), new StringToken("A")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.5"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.5"));
         assert primaryRanges.size() == 8;
         assert primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("D")));
         assert primaryRanges.contains(new Range<Token>(new StringToken("E"), new StringToken("F")));
@@ -418,23 +418,23 @@ public class StorageServiceServerTest
         metadata.clearUnsafe();
 
         // DC1
-        Multimap<InetAddress, Token> dc1 = HashMultimap.create();
-        dc1.put(InetAddress.getByName("127.0.0.1"), new StringToken("A"));
-        dc1.put(InetAddress.getByName("127.0.0.1"), new StringToken("E"));
-        dc1.put(InetAddress.getByName("127.0.0.1"), new StringToken("H"));
-        dc1.put(InetAddress.getByName("127.0.0.2"), new StringToken("C"));
-        dc1.put(InetAddress.getByName("127.0.0.2"), new StringToken("I"));
-        dc1.put(InetAddress.getByName("127.0.0.2"), new StringToken("J"));
+        Multimap<InetAddressAndPort, Token> dc1 = HashMultimap.create();
+        dc1.put(InetAddressAndPort.getByName("127.0.0.1"), new StringToken("A"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.1"), new StringToken("E"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.1"), new StringToken("H"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.2"), new StringToken("C"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.2"), new StringToken("I"));
+        dc1.put(InetAddressAndPort.getByName("127.0.0.2"), new StringToken("J"));
         metadata.updateNormalTokens(dc1);
 
         // DC2
-        Multimap<InetAddress, Token> dc2 = HashMultimap.create();
-        dc2.put(InetAddress.getByName("127.0.0.4"), new StringToken("B"));
-        dc2.put(InetAddress.getByName("127.0.0.4"), new StringToken("G"));
-        dc2.put(InetAddress.getByName("127.0.0.4"), new StringToken("L"));
-        dc2.put(InetAddress.getByName("127.0.0.5"), new StringToken("D"));
-        dc2.put(InetAddress.getByName("127.0.0.5"), new StringToken("F"));
-        dc2.put(InetAddress.getByName("127.0.0.5"), new StringToken("K"));
+        Multimap<InetAddressAndPort, Token> dc2 = HashMultimap.create();
+        dc2.put(InetAddressAndPort.getByName("127.0.0.4"), new StringToken("B"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.4"), new StringToken("G"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.4"), new StringToken("L"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.5"), new StringToken("D"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.5"), new StringToken("F"));
+        dc2.put(InetAddressAndPort.getByName("127.0.0.5"), new StringToken("K"));
         metadata.updateNormalTokens(dc2);
 
         Map<String, String> configOptions = new HashMap<>();
@@ -447,7 +447,7 @@ public class StorageServiceServerTest
         Schema.instance.load(meta);
 
         // endpoints in DC1 should have primary ranges which also cover DC2
-        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.1"));
+        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.1"));
         assertEquals(8, primaryRanges.size());
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("J"), new StringToken("K"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("K"), new StringToken("L"))));
@@ -459,7 +459,7 @@ public class StorageServiceServerTest
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("G"), new StringToken("H"))));
 
         // endpoints in DC1 should have primary ranges which also cover DC2
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.2"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.2"));
         assertEquals(4, primaryRanges.size());
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("B"), new StringToken("C"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B"))));
@@ -467,7 +467,7 @@ public class StorageServiceServerTest
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("I"), new StringToken("J"))));
 
         // endpoints in DC2 should have primary ranges which also cover DC1
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.4"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.4"));
         assertEquals(4, primaryRanges.size());
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("F"), new StringToken("G"))));
@@ -476,7 +476,7 @@ public class StorageServiceServerTest
         // the node covers range (L, A]
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("L"), new StringToken("A"))));
 
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.5"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.5"));
         assertTrue(primaryRanges.size() == 8);
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("D"))));
         assertTrue(primaryRanges.contains(new Range<Token>(new StringToken("E"), new StringToken("F"))));
@@ -497,23 +497,23 @@ public class StorageServiceServerTest
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
 
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.2"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.3"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.3"));
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.simpleTransient(2));
         Schema.instance.load(meta);
 
-        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
+        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.1"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("A")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.2"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.2"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.3"));
+        primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddressAndPort.getByName("127.0.0.3"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("B"), new StringToken("C")));
     }
@@ -525,9 +525,9 @@ public class StorageServiceServerTest
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
 
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.2"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.3"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.3"));
 
         Map<String, String> configOptions = new HashMap<>();
         configOptions.put("replication_factor", "2");
@@ -536,15 +536,15 @@ public class StorageServiceServerTest
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.simpleTransient(2));
         Schema.instance.load(meta);
 
-        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.1"));
+        Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.1"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("C"), new StringToken("A")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.2"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.2"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("A"), new StringToken("B")));
 
-        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.3"));
+        primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddressAndPort.getByName("127.0.0.3"));
         assert primaryRanges.size() == 1;
         assert primaryRanges.contains(new Range<Token>(new StringToken("B"), new StringToken("C")));
     }
@@ -557,10 +557,10 @@ public class StorageServiceServerTest
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
 
-        metadata.updateNormalToken(new LongToken(1000L), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new LongToken(2000L), InetAddress.getByName("127.0.0.2"));
-        metadata.updateNormalToken(new LongToken(3000L), InetAddress.getByName("127.0.0.3"));
-        metadata.updateNormalToken(new LongToken(4000L), InetAddress.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new LongToken(1000L), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new LongToken(2000L), InetAddressAndPort.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new LongToken(3000L), InetAddressAndPort.getByName("127.0.0.3"));
+        metadata.updateNormalToken(new LongToken(4000L), InetAddressAndPort.getByName("127.0.0.4"));
 
         Collection<Range<Token>> repairRangeFrom = StorageService.instance.createRepairRangeFrom("1500", "3700");
         assert repairRangeFrom.size() == 3;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
index 8172463..f8567e8 100644
--- a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
+++ b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
@@ -37,6 +37,7 @@ import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -48,7 +49,7 @@ public class WriteResponseHandlerTest
 {
     static Keyspace ks;
     static ColumnFamilyStore cfs;
-    static List<InetAddress> targets;
+    static List<InetAddressAndPort> targets;
 
     @BeforeClass
     public static void setUpClass() throws Throwable
@@ -57,36 +58,36 @@ public class WriteResponseHandlerTest
         // Register peers with expected DC for NetworkTopologyStrategy.
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
-        metadata.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.1.0.255"));
-        metadata.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.2.0.255"));
+        metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.1.0.255"));
+        metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.2.0.255"));
 
         DatabaseDescriptor.setEndpointSnitch(new IEndpointSnitch()
         {
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 return null;
             }
 
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
-                byte[] address = endpoint.getAddress();
+                byte[] address = endpoint.address.getAddress();
                 if (address[1] == 1)
                     return "datacenter1";
                 else
                     return "datacenter2";
             }
 
-            public List<InetAddress> getSortedListByProximity(InetAddress address, Collection<InetAddress> unsortedAddress)
+            public List<InetAddressAndPort> getSortedListByProximity(InetAddressAndPort address, Collection<InetAddressAndPort> unsortedAddress)
             {
                 return null;
             }
 
-            public void sortByProximity(InetAddress address, List<InetAddress> addresses)
+            public void sortByProximity(InetAddressAndPort address, List<InetAddressAndPort> addresses)
             {
 
             }
 
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
             {
                 return 0;
             }
@@ -96,7 +97,7 @@ public class WriteResponseHandlerTest
 
             }
 
-            public boolean isWorthMergingForRangeQuery(List<InetAddress> merged, List<InetAddress> l1, List<InetAddress> l2)
+            public boolean isWorthMergingForRangeQuery(List<InetAddressAndPort> merged, List<InetAddressAndPort> l1, List<InetAddressAndPort> l2)
             {
                 return false;
             }
@@ -105,8 +106,8 @@ public class WriteResponseHandlerTest
         SchemaLoader.createKeyspace("Foo", KeyspaceParams.nts("datacenter1", 3, "datacenter2", 3), SchemaLoader.standardCFMD("Foo", "Bar"));
         ks = Keyspace.open("Foo");
         cfs = ks.getColumnFamilyStore("Bar");
-        targets = ImmutableList.of(InetAddress.getByName("127.1.0.255"), InetAddress.getByName("127.1.0.254"), InetAddress.getByName("127.1.0.253"),
-                                   InetAddress.getByName("127.2.0.255"), InetAddress.getByName("127.2.0.254"), InetAddress.getByName("127.2.0.253"));
+        targets = ImmutableList.of(InetAddressAndPort.getByName("127.1.0.255"), InetAddressAndPort.getByName("127.1.0.254"), InetAddressAndPort.getByName("127.1.0.253"),
+                                   InetAddressAndPort.getByName("127.2.0.255"), InetAddressAndPort.getByName("127.2.0.254"), InetAddressAndPort.getByName("127.2.0.253"));
     }
 
     @Before

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java b/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
index bc24979..4f0c494 100644
--- a/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
+++ b/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 
 import org.junit.Test;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -36,7 +36,7 @@ public class SessionInfoTest
     public void testTotals()
     {
         TableId tableId = TableId.generate();
-        InetAddress local = FBUtilities.getLocalAddress();
+        InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
 
         Collection<StreamSummary> summaries = new ArrayList<>();
         for (int i = 0; i < 10; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 5c29698..ceaaae0 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -41,6 +40,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.messages.OutgoingFileMessage;
@@ -74,7 +74,7 @@ public class StreamTransferTaskTest
     @Test
     public void testScheduleTimeout() throws Exception
     {
-        InetAddress peer = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
         StreamSession session = new StreamSession(peer, peer, (connectionId, protocolVersion) -> new EmbeddedChannel(), 0, true, UUID.randomUUID(), PreviewKind.ALL);
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
 
@@ -120,7 +120,7 @@ public class StreamTransferTaskTest
     @Test
     public void testFailSessionDuringTransferShouldNotReleaseReferences() throws Exception
     {
-        InetAddress peer = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
         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);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 7a51d0c..16c07a0 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
@@ -35,6 +34,7 @@ import junit.framework.Assert;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
@@ -69,7 +69,7 @@ public class StreamingTransferTest
         DatabaseDescriptor.daemonInitialization();
     }
 
-    public static final InetAddress LOCAL = FBUtilities.getBroadcastAddress();
+    public static final InetAddressAndPort LOCAL = FBUtilities.getBroadcastAddressAndPort();
     public static final String KEYSPACE1 = "StreamingTransferTest1";
     public static final String CF_STANDARD = "Standard1";
     public static final String CF_COUNTER = "Counter1";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index a9849a3..617bae1 100644
--- a/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java
+++ b/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java
@@ -25,6 +25,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import com.google.common.net.InetAddresses;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -34,6 +35,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.async.TestScheduledFuture;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.StreamOperation;
@@ -43,7 +45,7 @@ import org.apache.cassandra.streaming.messages.CompleteMessage;
 
 public class NettyStreamingMessageSenderTest
 {
-    private static final InetSocketAddress REMOTE_ADDR = new InetSocketAddress("127.0.0.2", 0);
+    private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0);
 
     private EmbeddedChannel channel;
     private StreamSession session;
@@ -62,8 +64,8 @@ public class NettyStreamingMessageSenderTest
         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 = new StreamSession(REMOTE_ADDR, REMOTE_ADDR, (connectionId, protocolVersion) -> null, 0, true, pendingRepair, PreviewKind.ALL);
+        StreamResultFuture future = StreamResultFuture.initReceivingSide(0, UUID.randomUUID(), StreamOperation.REPAIR, REMOTE_ADDR, channel, true, pendingRepair, session.getPreviewKind());
         session.init(future);
         sender = session.getMessageSender();
         sender.setControlMessageChannel(channel);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index a674e6b..b4a736e 100644
--- a/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java
+++ b/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java
@@ -24,6 +24,7 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.UUID;
 
+import com.google.common.net.InetAddresses;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -35,6 +36,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -52,7 +54,7 @@ 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 static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0);
 
     private StreamingInboundHandler handler;
     private EmbeddedChannel channel;
@@ -123,7 +125,7 @@ public class StreamingInboundHandlerTest
     @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);
+        StreamInitMessage msg = new StreamInitMessage(REMOTE_ADDR, 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);
@@ -145,7 +147,7 @@ public class StreamingInboundHandlerTest
     @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,
+        FileMessageHeader header = new FileMessageHeader(TableId.generate(), REMOTE_ADDR, 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);
@@ -156,9 +158,9 @@ public class StreamingInboundHandlerTest
     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);
+        StreamResultFuture future = StreamResultFuture.initReceivingSide(0, planId, StreamOperation.REPAIR, REMOTE_ADDR, channel, true, UUID.randomUUID(), PreviewKind.ALL);
         StreamManager.instance.register(future);
-        FileMessageHeader header = new FileMessageHeader(TableId.generate(), REMOTE_ADDR.getAddress(), planId, 0, 0,
+        FileMessageHeader header = new FileMessageHeader(TableId.generate(), REMOTE_ADDR, 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);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/tracing/TracingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tracing/TracingTest.java b/test/unit/org/apache/cassandra/tracing/TracingTest.java
index f546496..61e08b0 100644
--- a/test/unit/org/apache/cassandra/tracing/TracingTest.java
+++ b/test/unit/org/apache/cassandra/tracing/TracingTest.java
@@ -31,6 +31,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.progress.ProgressEvent;
 import org.apache.commons.lang3.StringUtils;
 
@@ -197,7 +198,7 @@ public final class TracingTest
             return super.newSession(sessionId, traceType, customPayload);
         }
 
-        protected TraceState newTraceState(InetAddress ia, UUID uuid, Tracing.TraceType tt)
+        protected TraceState newTraceState(InetAddressAndPort ia, UUID uuid, Tracing.TraceType tt)
         {
             return new TraceState(ia, uuid, tt)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java b/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java
index 2dcd2ac..3d47ff3 100644
--- a/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java
+++ b/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.transport;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.HashMap;
 import java.util.Map;
@@ -33,26 +32,27 @@ import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.exceptions.WriteFailureException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.transport.messages.ErrorMessage;
 
 import static org.junit.Assert.assertEquals;
 
 public class ErrorMessageTest
 {
-    private static Map<InetAddress, RequestFailureReason> failureReasonMap1;
-    private static Map<InetAddress, RequestFailureReason> failureReasonMap2;
+    private static Map<InetAddressAndPort, RequestFailureReason> failureReasonMap1;
+    private static Map<InetAddressAndPort, RequestFailureReason> failureReasonMap2;
 
     @BeforeClass
     public static void setUpFixtures() throws UnknownHostException
     {
         failureReasonMap1 = new HashMap<>();
-        failureReasonMap1.put(InetAddress.getByName("127.0.0.1"), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
-        failureReasonMap1.put(InetAddress.getByName("127.0.0.2"), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
-        failureReasonMap1.put(InetAddress.getByName("127.0.0.3"), RequestFailureReason.UNKNOWN);
+        failureReasonMap1.put(InetAddressAndPort.getByName("127.0.0.1"), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
+        failureReasonMap1.put(InetAddressAndPort.getByName("127.0.0.2"), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
+        failureReasonMap1.put(InetAddressAndPort.getByName("127.0.0.3"), RequestFailureReason.UNKNOWN);
 
         failureReasonMap2 = new HashMap<>();
-        failureReasonMap2.put(InetAddress.getByName("127.0.0.1"), RequestFailureReason.UNKNOWN);
-        failureReasonMap2.put(InetAddress.getByName("127.0.0.2"), RequestFailureReason.UNKNOWN);
+        failureReasonMap2.put(InetAddressAndPort.getByName("127.0.0.1"), RequestFailureReason.UNKNOWN);
+        failureReasonMap2.put(InetAddressAndPort.getByName("127.0.0.2"), RequestFailureReason.UNKNOWN);
     }
 
     @Test
@@ -102,11 +102,11 @@ public class ErrorMessageTest
     @Test
     public void testRequestFailureExceptionMakesCopy() throws UnknownHostException
     {
-        Map<InetAddress, RequestFailureReason> modifiableFailureReasons = new HashMap<>(failureReasonMap1);
+        Map<InetAddressAndPort, RequestFailureReason> modifiableFailureReasons = new HashMap<>(failureReasonMap1);
         ReadFailureException rfe = new ReadFailureException(ConsistencyLevel.ALL, 3, 3, false, modifiableFailureReasons);
         WriteFailureException wfe = new WriteFailureException(ConsistencyLevel.ALL, 3, 3, WriteType.SIMPLE, modifiableFailureReasons);
 
-        modifiableFailureReasons.put(InetAddress.getByName("127.0.0.4"), RequestFailureReason.UNKNOWN);
+        modifiableFailureReasons.put(InetAddressAndPort.getByName("127.0.0.4"), RequestFailureReason.UNKNOWN);
 
         assertEquals(failureReasonMap1, rfe.failureReasonByEndpoint);
         assertEquals(failureReasonMap1, wfe.failureReasonByEndpoint);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/transport/ProtocolVersionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/transport/ProtocolVersionTest.java b/test/unit/org/apache/cassandra/transport/ProtocolVersionTest.java
index e287c08..6b95c67 100644
--- a/test/unit/org/apache/cassandra/transport/ProtocolVersionTest.java
+++ b/test/unit/org/apache/cassandra/transport/ProtocolVersionTest.java
@@ -62,7 +62,8 @@ public class ProtocolVersionTest
         Assert.assertNotNull(ProtocolVersion.CURRENT);
 
         Assert.assertFalse(ProtocolVersion.V4.isBeta());
-        Assert.assertTrue(ProtocolVersion.V5.isBeta());
+        Assert.assertFalse(ProtocolVersion.V5.isBeta());
+        Assert.assertTrue(ProtocolVersion.V6.isBeta());
     }
 
     @Test
@@ -72,24 +73,30 @@ public class ProtocolVersionTest
         Assert.assertTrue(ProtocolVersion.V2.isSmallerOrEqualTo(ProtocolVersion.V2));
         Assert.assertTrue(ProtocolVersion.V3.isSmallerOrEqualTo(ProtocolVersion.V3));
         Assert.assertTrue(ProtocolVersion.V4.isSmallerOrEqualTo(ProtocolVersion.V4));
+        Assert.assertTrue(ProtocolVersion.V5.isSmallerOrEqualTo(ProtocolVersion.V5));
 
         Assert.assertTrue(ProtocolVersion.V1.isGreaterOrEqualTo(ProtocolVersion.V1));
         Assert.assertTrue(ProtocolVersion.V2.isGreaterOrEqualTo(ProtocolVersion.V2));
         Assert.assertTrue(ProtocolVersion.V3.isGreaterOrEqualTo(ProtocolVersion.V3));
         Assert.assertTrue(ProtocolVersion.V4.isGreaterOrEqualTo(ProtocolVersion.V4));
+        Assert.assertTrue(ProtocolVersion.V5.isGreaterOrEqualTo(ProtocolVersion.V5));
 
         Assert.assertTrue(ProtocolVersion.V1.isSmallerThan(ProtocolVersion.V2));
         Assert.assertTrue(ProtocolVersion.V2.isSmallerThan(ProtocolVersion.V3));
         Assert.assertTrue(ProtocolVersion.V3.isSmallerThan(ProtocolVersion.V4));
+        Assert.assertTrue(ProtocolVersion.V4.isSmallerThan(ProtocolVersion.V5));
 
         Assert.assertFalse(ProtocolVersion.V1.isGreaterThan(ProtocolVersion.V2));
         Assert.assertFalse(ProtocolVersion.V2.isGreaterThan(ProtocolVersion.V3));
         Assert.assertFalse(ProtocolVersion.V3.isGreaterThan(ProtocolVersion.V4));
+        Assert.assertFalse(ProtocolVersion.V4.isGreaterThan(ProtocolVersion.V5));
 
+        Assert.assertTrue(ProtocolVersion.V5.isGreaterThan(ProtocolVersion.V4));
         Assert.assertTrue(ProtocolVersion.V4.isGreaterThan(ProtocolVersion.V3));
         Assert.assertTrue(ProtocolVersion.V3.isGreaterThan(ProtocolVersion.V2));
         Assert.assertTrue(ProtocolVersion.V2.isGreaterThan(ProtocolVersion.V1));
 
+        Assert.assertFalse(ProtocolVersion.V5.isSmallerThan(ProtocolVersion.V4));
         Assert.assertFalse(ProtocolVersion.V4.isSmallerThan(ProtocolVersion.V3));
         Assert.assertFalse(ProtocolVersion.V3.isSmallerThan(ProtocolVersion.V2));
         Assert.assertFalse(ProtocolVersion.V2.isSmallerThan(ProtocolVersion.V1));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/transport/SerDeserTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/transport/SerDeserTest.java b/test/unit/org/apache/cassandra/transport/SerDeserTest.java
index 2592ae7..d3b9282 100644
--- a/test/unit/org/apache/cassandra/transport/SerDeserTest.java
+++ b/test/unit/org/apache/cassandra/transport/SerDeserTest.java
@@ -42,12 +42,14 @@ import static org.junit.Assert.assertEquals;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Serialization/deserialization tests for protocol objects and messages.
  */
 public class SerDeserTest
 {
+
     @BeforeClass
     public static void setupDD()
     {
@@ -113,12 +115,12 @@ public class SerDeserTest
     {
         List<Event> events = new ArrayList<>();
 
-        events.add(TopologyChange.newNode(FBUtilities.getBroadcastAddress(), 42));
-        events.add(TopologyChange.removedNode(FBUtilities.getBroadcastAddress(), 42));
-        events.add(TopologyChange.movedNode(FBUtilities.getBroadcastAddress(), 42));
+        events.add(TopologyChange.newNode(FBUtilities.getBroadcastAddressAndPort()));
+        events.add(TopologyChange.removedNode(FBUtilities.getBroadcastAddressAndPort()));
+        events.add(TopologyChange.movedNode(FBUtilities.getBroadcastAddressAndPort()));
 
-        events.add(StatusChange.nodeUp(FBUtilities.getBroadcastAddress(), 42));
-        events.add(StatusChange.nodeDown(FBUtilities.getBroadcastAddress(), 42));
+        events.add(StatusChange.nodeUp(FBUtilities.getBroadcastAddressAndPort()));
+        events.add(StatusChange.nodeDown(FBUtilities.getBroadcastAddressAndPort()));
 
         events.add(new SchemaChange(SchemaChange.Change.CREATED, "ks"));
         events.add(new SchemaChange(SchemaChange.Change.UPDATED, "ks"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
index b8d2633..a982624 100644
--- a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
+++ b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
@@ -153,7 +153,7 @@ public class FBUtilitiesTest
     }
 
     @Test
-    public void testGetBroadcastRpcAddress() throws Exception
+    public void testGetBroadcastNativeAddress() throws Exception
     {
         //When both rpc_address and broadcast_rpc_address are null, it should return the local address (from DD.applyAddressConfig)
         FBUtilities.reset();
@@ -161,21 +161,21 @@ public class FBUtilitiesTest
         testConfig.rpc_address = null;
         testConfig.broadcast_rpc_address = null;
         DatabaseDescriptor.applyAddressConfig(testConfig);
-        assertEquals(FBUtilities.getLocalAddress(), FBUtilities.getBroadcastRpcAddress());
+        assertEquals(FBUtilities.getJustLocalAddress(), FBUtilities.getJustBroadcastNativeAddress());
 
         //When rpc_address is defined and broadcast_rpc_address is null, it should return the rpc_address
         FBUtilities.reset();
         testConfig.rpc_address = "127.0.0.2";
         testConfig.broadcast_rpc_address = null;
         DatabaseDescriptor.applyAddressConfig(testConfig);
-        assertEquals(InetAddress.getByName("127.0.0.2"), FBUtilities.getBroadcastRpcAddress());
+        assertEquals(InetAddress.getByName("127.0.0.2"), FBUtilities.getJustBroadcastNativeAddress());
 
         //When both rpc_address and broadcast_rpc_address are defined, it should return broadcast_rpc_address
         FBUtilities.reset();
         testConfig.rpc_address = "127.0.0.2";
         testConfig.broadcast_rpc_address = "127.0.0.3";
         DatabaseDescriptor.applyAddressConfig(testConfig);
-        assertEquals(InetAddress.getByName("127.0.0.3"), FBUtilities.getBroadcastRpcAddress());
+        assertEquals(InetAddress.getByName("127.0.0.3"), FBUtilities.getJustBroadcastNativeAddress());
 
         FBUtilities.reset();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
index 867918a..ffc1ace 100644
--- a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
+++ b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.stress;
 
 import java.io.File;
 import java.io.IOError;
-import java.net.InetAddress;
 import java.net.URI;
 import java.util.*;
 import java.util.concurrent.*;
@@ -44,6 +43,7 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.stress.generate.PartitionGenerator;
@@ -185,7 +185,7 @@ public abstract class CompactionStress implements Runnable
         tokenMetadata.clearUnsafe();
         for (int i = 1; i <= numTokens; i++)
         {
-            InetAddress addr = FBUtilities.getBroadcastAddress();
+            InetAddressAndPort addr = FBUtilities.getBroadcastAddressAndPort();
             List<Token> tokens = Lists.newArrayListWithCapacity(numTokens);
             for (int j = 0; j < numTokens; ++j)
                 tokens.add(p.getRandomToken(random));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
index a029162..24c10bf 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
@@ -29,6 +29,8 @@ import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 
+import com.google.common.net.HostAndPort;
+
 import com.datastax.driver.core.Host;
 import org.apache.cassandra.stress.util.ResultLogger;
 
@@ -37,6 +39,7 @@ public class SettingsNode implements Serializable
     public final List<String> nodes;
     public final boolean isWhiteList;
     public final String datacenter;
+    public final boolean allowServerPortDiscovery;
 
     public SettingsNode(Options options)
     {
@@ -69,6 +72,7 @@ public class SettingsNode implements Serializable
 
         isWhiteList = options.whitelist.setByUser();
         datacenter = options.datacenter.value();
+        allowServerPortDiscovery = options.allowServerPortDiscovery.setByUser();
     }
 
     public Set<String> resolveAllPermitted(StressSettings settings)
@@ -80,7 +84,7 @@ public class SettingsNode implements Serializable
                 if (!isWhiteList)
                 {
                     for (Host host : settings.getJavaDriverClient().getCluster().getMetadata().getAllHosts())
-                        r.add(host.getAddress().getHostName());
+                        r.add(host.getSocketAddress().getHostString() + ":" + host.getSocketAddress().getPort());
                     break;
                 }
             case SIMPLE_NATIVE:
@@ -97,7 +101,8 @@ public class SettingsNode implements Serializable
         {
             try
             {
-                r.add(InetAddress.getByName(node));
+                HostAndPort hap = HostAndPort.fromString(node);
+                r.add(InetAddress.getByName(hap.getHost()));
             }
             catch (UnknownHostException e)
             {
@@ -114,7 +119,8 @@ public class SettingsNode implements Serializable
         {
             try
             {
-                r.add(new InetSocketAddress(InetAddress.getByName(node), port));
+                HostAndPort hap = HostAndPort.fromString(node).withDefaultPort(port);
+                r.add(new InetSocketAddress(InetAddress.getByName(hap.getHost()), hap.getPort()));
             }
             catch (UnknownHostException e)
             {
@@ -139,12 +145,13 @@ public class SettingsNode implements Serializable
         final OptionSimple datacenter = new OptionSimple("datacenter=", ".*", null, "Datacenter used for DCAwareRoundRobinLoadPolicy", false);
         final OptionSimple whitelist = new OptionSimple("whitelist", "", null, "Limit communications to the provided nodes", false);
         final OptionSimple file = new OptionSimple("file=", ".*", null, "Node file (one per line)", false);
+        final OptionSimple allowServerPortDiscovery = new OptionSimple("allow_server_port_discovery", "", null, "Allow Java client to discover server client port numbers", false);
         final OptionSimple list = new OptionSimple("", "[^=,]+(,[^=,]+)*", "localhost", "comma delimited list of nodes", false);
 
         @Override
         public List<? extends Option> options()
         {
-            return Arrays.asList(datacenter, whitelist, file, list);
+            return Arrays.asList(datacenter, whitelist, file, allowServerPortDiscovery, list);
         }
     }
 
@@ -154,6 +161,7 @@ public class SettingsNode implements Serializable
         out.println("  Nodes: " + nodes);
         out.println("  Is White List: " + isWhiteList);
         out.println("  Datacenter: " + datacenter);
+        out.println("  Allow server port discovery: " + allowServerPortDiscovery);
     }
 
     public static SettingsNode get(Map<String, String[]> clArgs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
index 4928cd2..fbcab4b 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
@@ -54,6 +54,7 @@ public class JavaDriverClient
     private Cluster cluster;
     private Session session;
     private final LoadBalancingPolicy loadBalancingPolicy;
+    private final boolean allowServerPortDiscovery;
 
     private static final ConcurrentMap<String, PreparedStatement> stmts = new ConcurrentHashMap<>();
 
@@ -73,6 +74,7 @@ public class JavaDriverClient
         this.encryptionOptions = encryptionOptions;
         this.loadBalancingPolicy = loadBalancingPolicy(settings);
         this.connectionsPerHost = settings.mode.connectionsPerHost == null ? 8 : settings.mode.connectionsPerHost;
+        this.allowServerPortDiscovery = settings.node.allowServerPortDiscovery;
 
         int maxThreadCount = 0;
         if (settings.rate.auto)
@@ -134,6 +136,9 @@ public class JavaDriverClient
                                                 .withoutJMXReporting()
                                                 .withProtocolVersion(protocolVersion)
                                                 .withoutMetrics(); // The driver uses metrics 3 with conflict with our version
+        if (allowServerPortDiscovery)
+            clusterBuilder = clusterBuilder.allowServerPortDiscovery();
+
         if (loadBalancingPolicy != null)
             clusterBuilder.withLoadBalancingPolicy(loadBalancingPolicy);
         clusterBuilder.withCompression(compression);
@@ -166,7 +171,7 @@ public class JavaDriverClient
         for (Host host : metadata.getAllHosts())
         {
             System.out.printf("Datacenter: %s; Host: %s; Rack: %s%n",
-                    host.getDatacenter(), host.getAddress(), host.getRack());
+                    host.getDatacenter(), host.getAddress() + ":" + host.getSocketAddress().getPort(), host.getRack());
         }
 
         session = cluster.connect();


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


[10/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index be0cf0f..dcf0cab 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -17,9 +17,7 @@
  */
 package org.apache.cassandra.service;
 
-import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.nio.file.Paths;
 import java.util.*;
@@ -64,7 +62,6 @@ import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.hints.Hint;
 import org.apache.cassandra.hints.HintsService;
 import org.apache.cassandra.index.Index;
-import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.*;
 import org.apache.cassandra.metrics.*;
 import org.apache.cassandra.net.*;
@@ -95,9 +92,9 @@ public class StorageProxy implements StorageProxyMBean
     public static final StorageProxy instance = new StorageProxy();
 
     private static volatile int maxHintsInProgress = 128 * FBUtilities.getAvailableProcessors();
-    private static final CacheLoader<InetAddress, AtomicInteger> hintsInProgress = new CacheLoader<InetAddress, AtomicInteger>()
+    private static final CacheLoader<InetAddressAndPort, AtomicInteger> hintsInProgress = new CacheLoader<InetAddressAndPort, AtomicInteger>()
     {
-        public AtomicInteger load(InetAddress inetAddress)
+        public AtomicInteger load(InetAddressAndPort inetAddress)
         {
             return new AtomicInteger(0);
         }
@@ -135,7 +132,7 @@ public class StorageProxy implements StorageProxyMBean
         standardWritePerformer = new WritePerformer()
         {
             public void apply(IMutation mutation,
-                              Iterable<InetAddress> targets,
+                              Iterable<InetAddressAndPort> targets,
                               AbstractWriteResponseHandler<IMutation> responseHandler,
                               String localDataCenter,
                               ConsistencyLevel consistency_level)
@@ -155,7 +152,7 @@ public class StorageProxy implements StorageProxyMBean
         counterWritePerformer = new WritePerformer()
         {
             public void apply(IMutation mutation,
-                              Iterable<InetAddress> targets,
+                              Iterable<InetAddressAndPort> targets,
                               AbstractWriteResponseHandler<IMutation> responseHandler,
                               String localDataCenter,
                               ConsistencyLevel consistencyLevel)
@@ -167,7 +164,7 @@ public class StorageProxy implements StorageProxyMBean
         counterWriteOnCoordinatorPerformer = new WritePerformer()
         {
             public void apply(IMutation mutation,
-                              Iterable<InetAddress> targets,
+                              Iterable<InetAddressAndPort> targets,
                               AbstractWriteResponseHandler<IMutation> responseHandler,
                               String localDataCenter,
                               ConsistencyLevel consistencyLevel)
@@ -248,8 +245,8 @@ public class StorageProxy implements StorageProxyMBean
             while (System.nanoTime() - queryStartNanoTime < timeout)
             {
                 // for simplicity, we'll do a single liveness check at the start of each attempt
-                Pair<List<InetAddress>, Integer> p = getPaxosParticipants(metadata, key, consistencyForPaxos);
-                List<InetAddress> liveEndpoints = p.left;
+                Pair<List<InetAddressAndPort>, Integer> p = getPaxosParticipants(metadata, key, consistencyForPaxos);
+                List<InetAddressAndPort> liveEndpoints = p.left;
                 int requiredParticipants = p.right;
 
                 final Pair<UUID, Integer> pair = beginAndRepairPaxos(queryStartNanoTime, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos, consistencyForCommit, true, state);
@@ -342,34 +339,34 @@ public class StorageProxy implements StorageProxyMBean
             casWriteMetrics.contention.update(contentions);
     }
 
-    private static Predicate<InetAddress> sameDCPredicateFor(final String dc)
+    private static Predicate<InetAddressAndPort> sameDCPredicateFor(final String dc)
     {
         final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        return new Predicate<InetAddress>()
+        return new Predicate<InetAddressAndPort>()
         {
-            public boolean apply(InetAddress host)
+            public boolean apply(InetAddressAndPort host)
             {
                 return dc.equals(snitch.getDatacenter(host));
             }
         };
     }
 
-    private static Pair<List<InetAddress>, Integer> getPaxosParticipants(TableMetadata metadata, DecoratedKey key, ConsistencyLevel consistencyForPaxos) throws UnavailableException
+    private static Pair<List<InetAddressAndPort>, Integer> getPaxosParticipants(TableMetadata metadata, DecoratedKey key, ConsistencyLevel consistencyForPaxos) throws UnavailableException
     {
         Token tk = key.getToken();
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(metadata.keyspace, tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, metadata.keyspace);
+        List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(metadata.keyspace, tk);
+        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, metadata.keyspace);
         if (consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL)
         {
             // Restrict naturalEndpoints and pendingEndpoints to node in the local DC only
-            String localDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
-            Predicate<InetAddress> isLocalDc = sameDCPredicateFor(localDc);
+            String localDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
+            Predicate<InetAddressAndPort> isLocalDc = sameDCPredicateFor(localDc);
             naturalEndpoints = ImmutableList.copyOf(Iterables.filter(naturalEndpoints, isLocalDc));
             pendingEndpoints = ImmutableList.copyOf(Iterables.filter(pendingEndpoints, isLocalDc));
         }
         int participants = pendingEndpoints.size() + naturalEndpoints.size();
         int requiredParticipants = participants / 2 + 1; // See CASSANDRA-8346, CASSANDRA-833
-        List<InetAddress> liveEndpoints = ImmutableList.copyOf(Iterables.filter(Iterables.concat(naturalEndpoints, pendingEndpoints), IAsyncCallback.isAlive));
+        List<InetAddressAndPort> liveEndpoints = ImmutableList.copyOf(Iterables.filter(Iterables.concat(naturalEndpoints, pendingEndpoints), IAsyncCallback.isAlive));
         if (liveEndpoints.size() < requiredParticipants)
             throw new UnavailableException(consistencyForPaxos, requiredParticipants, liveEndpoints.size());
 
@@ -394,7 +391,7 @@ public class StorageProxy implements StorageProxyMBean
     private static Pair<UUID, Integer> beginAndRepairPaxos(long queryStartNanoTime,
                                                            DecoratedKey key,
                                                            TableMetadata metadata,
-                                                           List<InetAddress> liveEndpoints,
+                                                           List<InetAddressAndPort> liveEndpoints,
                                                            int requiredParticipants,
                                                            ConsistencyLevel consistencyForPaxos,
                                                            ConsistencyLevel consistencyForCommit,
@@ -472,7 +469,7 @@ public class StorageProxy implements StorageProxyMBean
             // Since we waited for quorum nodes, if some of them haven't seen the last commit (which may just be a timing issue, but may also
             // mean we lost messages), we pro-actively "repair" those nodes, and retry.
             int nowInSec = Ints.checkedCast(TimeUnit.MICROSECONDS.toSeconds(ballotMicros));
-            Iterable<InetAddress> missingMRC = summary.replicasMissingMostRecentCommit(metadata, nowInSec);
+            Iterable<InetAddressAndPort> missingMRC = summary.replicasMissingMostRecentCommit(metadata, nowInSec);
             if (Iterables.size(missingMRC) > 0)
             {
                 Tracing.trace("Repairing replicas that missed the most recent commit");
@@ -494,19 +491,19 @@ public class StorageProxy implements StorageProxyMBean
     /**
      * Unlike commitPaxos, this does not wait for replies
      */
-    private static void sendCommit(Commit commit, Iterable<InetAddress> replicas)
+    private static void sendCommit(Commit commit, Iterable<InetAddressAndPort> replicas)
     {
         MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_COMMIT, commit, Commit.serializer);
-        for (InetAddress target : replicas)
+        for (InetAddressAndPort target : replicas)
             MessagingService.instance().sendOneWay(message, target);
     }
 
-    private static PrepareCallback preparePaxos(Commit toPrepare, List<InetAddress> endpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos, long queryStartNanoTime)
+    private static PrepareCallback preparePaxos(Commit toPrepare, List<InetAddressAndPort> endpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos, long queryStartNanoTime)
     throws WriteTimeoutException
     {
         PrepareCallback callback = new PrepareCallback(toPrepare.update.partitionKey(), toPrepare.update.metadata(), requiredParticipants, consistencyForPaxos, queryStartNanoTime);
         MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_PREPARE, toPrepare, Commit.serializer);
-        for (InetAddress target : endpoints)
+        for (InetAddressAndPort target : endpoints)
         {
             if (canDoLocalRequest(target))
             {
@@ -516,9 +513,9 @@ public class StorageProxy implements StorageProxyMBean
                     {
                         try
                         {
-                            MessageIn<PrepareResponse> message = MessageIn.create(FBUtilities.getBroadcastAddress(),
+                            MessageIn<PrepareResponse> message = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(),
                                     PrepareVerbHandler.doPrepare(toPrepare),
-                                    Collections.<String, byte[]>emptyMap(),
+                                    Collections.emptyMap(),
                                     MessagingService.Verb.INTERNAL_RESPONSE,
                                     MessagingService.current_version);
                             callback.response(message);
@@ -539,12 +536,12 @@ public class StorageProxy implements StorageProxyMBean
         return callback;
     }
 
-    private static boolean proposePaxos(Commit proposal, List<InetAddress> endpoints, int requiredParticipants, boolean timeoutIfPartial, ConsistencyLevel consistencyLevel, long queryStartNanoTime)
+    private static boolean proposePaxos(Commit proposal, List<InetAddressAndPort> endpoints, int requiredParticipants, boolean timeoutIfPartial, ConsistencyLevel consistencyLevel, long queryStartNanoTime)
     throws WriteTimeoutException
     {
         ProposeCallback callback = new ProposeCallback(endpoints.size(), requiredParticipants, !timeoutIfPartial, consistencyLevel, queryStartNanoTime);
         MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_PROPOSE, proposal, Commit.serializer);
-        for (InetAddress target : endpoints)
+        for (InetAddressAndPort target : endpoints)
         {
             if (canDoLocalRequest(target))
             {
@@ -554,9 +551,9 @@ public class StorageProxy implements StorageProxyMBean
                     {
                         try
                         {
-                            MessageIn<Boolean> message = MessageIn.create(FBUtilities.getBroadcastAddress(),
+                            MessageIn<Boolean> message = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(),
                                     ProposeVerbHandler.doPropose(proposal),
-                                    Collections.<String, byte[]>emptyMap(),
+                                    Collections.emptyMap(),
                                     MessagingService.Verb.INTERNAL_RESPONSE,
                                     MessagingService.current_version);
                             callback.response(message);
@@ -590,8 +587,8 @@ public class StorageProxy implements StorageProxyMBean
         Keyspace keyspace = Keyspace.open(proposal.update.metadata().keyspace);
 
         Token tk = proposal.update.partitionKey().getToken();
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspace.getName(), tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspace.getName());
+        List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspace.getName(), tk);
+        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspace.getName());
 
         AbstractWriteResponseHandler<Commit> responseHandler = null;
         if (shouldBlock)
@@ -602,7 +599,7 @@ public class StorageProxy implements StorageProxyMBean
         }
 
         MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_COMMIT, proposal, Commit.serializer);
-        for (InetAddress destination : Iterables.concat(naturalEndpoints, pendingEndpoints))
+        for (InetAddressAndPort destination : Iterables.concat(naturalEndpoints, pendingEndpoints))
         {
             checkHintOverload(destination);
 
@@ -658,7 +655,7 @@ public class StorageProxy implements StorageProxyMBean
                 {
                     if (!(ex instanceof WriteTimeoutException))
                         logger.error("Failed to apply paxos commit locally : ", ex);
-                    responseHandler.onFailure(FBUtilities.getBroadcastAddress(), RequestFailureReason.UNKNOWN);
+                    responseHandler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN);
                 }
             }
 
@@ -684,7 +681,7 @@ public class StorageProxy implements StorageProxyMBean
     throws UnavailableException, OverloadedException, WriteTimeoutException, WriteFailureException
     {
         Tracing.trace("Determining replicas for mutation");
-        final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
 
         long startTime = System.nanoTime();
         List<AbstractWriteResponseHandler<IMutation>> responseHandlers = new ArrayList<>(mutations.size());
@@ -781,13 +778,13 @@ public class StorageProxy implements StorageProxyMBean
         String keyspaceName = mutation.getKeyspaceName();
         Token token = mutation.key().getToken();
 
-        Iterable<InetAddress> endpoints = StorageService.instance.getNaturalAndPendingEndpoints(keyspaceName, token);
-        ArrayList<InetAddress> endpointsToHint = new ArrayList<>(Iterables.size(endpoints));
+        Iterable<InetAddressAndPort> endpoints = StorageService.instance.getNaturalAndPendingEndpoints(keyspaceName, token);
+        ArrayList<InetAddressAndPort> endpointsToHint = new ArrayList<>(Iterables.size(endpoints));
 
         // local writes can timeout, but cannot be dropped (see LocalMutationRunnable and CASSANDRA-6510),
         // so there is no need to hint or retry.
-        for (InetAddress target : endpoints)
-            if (!target.equals(FBUtilities.getBroadcastAddress()) && shouldHint(target))
+        for (InetAddressAndPort target : endpoints)
+            if (!target.equals(FBUtilities.getBroadcastAddressAndPort()) && shouldHint(target))
                 endpointsToHint.add(target);
 
         submitHint(mutation, endpointsToHint, null);
@@ -797,7 +794,7 @@ public class StorageProxy implements StorageProxyMBean
     {
         String keyspaceName = mutation.getKeyspaceName();
         Token token = mutation.key().getToken();
-        InetAddress local = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
 
         return StorageService.instance.getNaturalEndpoints(keyspaceName, token).contains(local)
                || StorageService.instance.getTokenMetadata().pendingEndpointsFor(token, keyspaceName).contains(local);
@@ -816,7 +813,7 @@ public class StorageProxy implements StorageProxyMBean
     throws UnavailableException, OverloadedException, WriteTimeoutException
     {
         Tracing.trace("Determining replicas for mutation");
-        final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
 
         long startTime = System.nanoTime();
 
@@ -841,7 +838,7 @@ public class StorageProxy implements StorageProxyMBean
                 ConsistencyLevel consistencyLevel = ConsistencyLevel.ONE;
 
                 //Since the base -> view replication is 1:1 we only need to store the BL locally
-                final Collection<InetAddress> batchlogEndpoints = Collections.singleton(FBUtilities.getBroadcastAddress());
+                final Collection<InetAddressAndPort> batchlogEndpoints = Collections.singleton(FBUtilities.getBroadcastAddressAndPort());
                 BatchlogResponseHandler.BatchlogCleanup cleanup = new BatchlogResponseHandler.BatchlogCleanup(mutations.size(), () -> asyncRemoveFromBatchlog(batchlogEndpoints, batchUUID));
 
                 // add a handler for each mutation - includes checking availability, but doesn't initiate any writes, yet
@@ -849,8 +846,8 @@ public class StorageProxy implements StorageProxyMBean
                 {
                     String keyspaceName = mutation.getKeyspaceName();
                     Token tk = mutation.key().getToken();
-                    Optional<InetAddress> pairedEndpoint = ViewUtils.getViewNaturalEndpoint(keyspaceName, baseToken, tk);
-                    Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
+                    Optional<InetAddressAndPort> pairedEndpoint = ViewUtils.getViewNaturalEndpoint(keyspaceName, baseToken, tk);
+                    Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
 
                     // if there are no paired endpoints there are probably range movements going on, so we write to the local batchlog to replay later
                     if (!pairedEndpoint.isPresent())
@@ -865,7 +862,7 @@ public class StorageProxy implements StorageProxyMBean
                     }
 
                     // When local node is the paired endpoint just apply the mutation locally.
-                    if (pairedEndpoint.get().equals(FBUtilities.getBroadcastAddress()) && StorageService.instance.isJoined())
+                    if (pairedEndpoint.get().equals(FBUtilities.getBroadcastAddressAndPort()) && StorageService.instance.isJoined())
                     {
                         try
                         {
@@ -956,7 +953,7 @@ public class StorageProxy implements StorageProxyMBean
         long startTime = System.nanoTime();
 
         List<WriteResponseHandlerWrapper> wrappers = new ArrayList<WriteResponseHandlerWrapper>(mutations.size());
-        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
 
         try
         {
@@ -974,7 +971,7 @@ public class StorageProxy implements StorageProxyMBean
                     batchConsistencyLevel = consistency_level;
             }
 
-            final Collection<InetAddress> batchlogEndpoints = getBatchlogEndpoints(localDataCenter, batchConsistencyLevel);
+            final Collection<InetAddressAndPort> batchlogEndpoints = getBatchlogEndpoints(localDataCenter, batchConsistencyLevel);
             final UUID batchUUID = UUIDGen.getTimeUUID();
             BatchlogResponseHandler.BatchlogCleanup cleanup = new BatchlogResponseHandler.BatchlogCleanup(mutations.size(),
                                                                                                           () -> asyncRemoveFromBatchlog(batchlogEndpoints, batchUUID));
@@ -1029,16 +1026,16 @@ public class StorageProxy implements StorageProxyMBean
         }
     }
 
-    public static boolean canDoLocalRequest(InetAddress replica)
+    public static boolean canDoLocalRequest(InetAddressAndPort replica)
     {
-        return replica.equals(FBUtilities.getBroadcastAddress());
+        return replica.equals(FBUtilities.getBroadcastAddressAndPort());
     }
 
-    private static void syncWriteToBatchlog(Collection<Mutation> mutations, Collection<InetAddress> endpoints, UUID uuid, long queryStartNanoTime)
+    private static void syncWriteToBatchlog(Collection<Mutation> mutations, Collection<InetAddressAndPort> endpoints, UUID uuid, long queryStartNanoTime)
     throws WriteTimeoutException, WriteFailureException
     {
         WriteResponseHandler<?> handler = new WriteResponseHandler<>(endpoints,
-                                                                     Collections.<InetAddress>emptyList(),
+                                                                     Collections.emptyList(),
                                                                      endpoints.size() == 1 ? ConsistencyLevel.ONE : ConsistencyLevel.TWO,
                                                                      Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME),
                                                                      null,
@@ -1047,7 +1044,7 @@ public class StorageProxy implements StorageProxyMBean
 
         Batch batch = Batch.createLocal(uuid, FBUtilities.timestampMicros(), mutations);
         MessageOut<Batch> message = new MessageOut<>(MessagingService.Verb.BATCH_STORE, batch, Batch.serializer);
-        for (InetAddress target : endpoints)
+        for (InetAddressAndPort target : endpoints)
         {
             logger.trace("Sending batchlog store request {} to {} for {} mutations", batch.id, target, batch.size());
 
@@ -1059,10 +1056,10 @@ public class StorageProxy implements StorageProxyMBean
         handler.get();
     }
 
-    private static void asyncRemoveFromBatchlog(Collection<InetAddress> endpoints, UUID uuid)
+    private static void asyncRemoveFromBatchlog(Collection<InetAddressAndPort> endpoints, UUID uuid)
     {
         MessageOut<UUID> message = new MessageOut<>(MessagingService.Verb.BATCH_REMOVE, uuid, UUIDSerializer.serializer);
-        for (InetAddress target : endpoints)
+        for (InetAddressAndPort target : endpoints)
         {
             if (logger.isTraceEnabled())
                 logger.trace("Sending batchlog remove request {} to {}", uuid, target);
@@ -1078,7 +1075,7 @@ public class StorageProxy implements StorageProxyMBean
     {
         for (WriteResponseHandlerWrapper wrapper : wrappers)
         {
-            Iterable<InetAddress> endpoints = Iterables.concat(wrapper.handler.naturalEndpoints, wrapper.handler.pendingEndpoints);
+            Iterable<InetAddressAndPort> endpoints = Iterables.concat(wrapper.handler.naturalEndpoints, wrapper.handler.pendingEndpoints);
 
             try
             {
@@ -1086,7 +1083,7 @@ public class StorageProxy implements StorageProxyMBean
             }
             catch (OverloadedException | WriteTimeoutException e)
             {
-                wrapper.handler.onFailure(FBUtilities.getBroadcastAddress(), RequestFailureReason.UNKNOWN);
+                wrapper.handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN);
             }
         }
     }
@@ -1096,7 +1093,7 @@ public class StorageProxy implements StorageProxyMBean
     {
         for (WriteResponseHandlerWrapper wrapper : wrappers)
         {
-            Iterable<InetAddress> endpoints = Iterables.concat(wrapper.handler.naturalEndpoints, wrapper.handler.pendingEndpoints);
+            Iterable<InetAddressAndPort> endpoints = Iterables.concat(wrapper.handler.naturalEndpoints, wrapper.handler.pendingEndpoints);
             sendToHintedEndpoints(wrapper.mutation, endpoints, wrapper.handler, localDataCenter, stage);
         }
 
@@ -1132,8 +1129,8 @@ public class StorageProxy implements StorageProxyMBean
         AbstractReplicationStrategy rs = Keyspace.open(keyspaceName).getReplicationStrategy();
 
         Token tk = mutation.key().getToken();
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
+        List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
+        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
 
         AbstractWriteResponseHandler<IMutation> responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, callback, writeType, queryStartNanoTime);
 
@@ -1156,8 +1153,8 @@ public class StorageProxy implements StorageProxyMBean
         AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
         String keyspaceName = mutation.getKeyspaceName();
         Token tk = mutation.key().getToken();
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
+        List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
+        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
         AbstractWriteResponseHandler<IMutation> writeHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, null, writeType, queryStartNanoTime);
         BatchlogResponseHandler<IMutation> batchHandler = new BatchlogResponseHandler<>(writeHandler, batchConsistencyLevel.blockFor(keyspace), cleanup, queryStartNanoTime);
         return new WriteResponseHandlerWrapper(batchHandler, mutation);
@@ -1170,7 +1167,7 @@ public class StorageProxy implements StorageProxyMBean
     private static WriteResponseHandlerWrapper wrapViewBatchResponseHandler(Mutation mutation,
                                                                             ConsistencyLevel consistency_level,
                                                                             ConsistencyLevel batchConsistencyLevel,
-                                                                            List<InetAddress> naturalEndpoints,
+                                                                            List<InetAddressAndPort> naturalEndpoints,
                                                                             AtomicLong baseComplete,
                                                                             WriteType writeType,
                                                                             BatchlogResponseHandler.BatchlogCleanup cleanup,
@@ -1180,7 +1177,7 @@ public class StorageProxy implements StorageProxyMBean
         AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
         String keyspaceName = mutation.getKeyspaceName();
         Token tk = mutation.key().getToken();
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
+        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
         AbstractWriteResponseHandler<IMutation> writeHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, () -> {
             long delay = Math.max(0, System.currentTimeMillis() - baseComplete.get());
             viewWriteMetrics.viewWriteLatency.update(delay, TimeUnit.MILLISECONDS);
@@ -1209,18 +1206,18 @@ public class StorageProxy implements StorageProxyMBean
      * - choose min(2, number of qualifying candiates above)
      * - allow the local node to be the only replica only if it's a single-node DC
      */
-    private static Collection<InetAddress> getBatchlogEndpoints(String localDataCenter, ConsistencyLevel consistencyLevel)
+    private static Collection<InetAddressAndPort> getBatchlogEndpoints(String localDataCenter, ConsistencyLevel consistencyLevel)
     throws UnavailableException
     {
         TokenMetadata.Topology topology = StorageService.instance.getTokenMetadata().cachedOnlyTokenMap().getTopology();
-        Multimap<String, InetAddress> localEndpoints = HashMultimap.create(topology.getDatacenterRacks().get(localDataCenter));
-        String localRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress());
+        Multimap<String, InetAddressAndPort> localEndpoints = HashMultimap.create(topology.getDatacenterRacks().get(localDataCenter));
+        String localRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddressAndPort());
 
-        Collection<InetAddress> chosenEndpoints = new BatchlogManager.EndpointFilter(localRack, localEndpoints).filter();
+        Collection<InetAddressAndPort> chosenEndpoints = new BatchlogManager.EndpointFilter(localRack, localEndpoints).filter();
         if (chosenEndpoints.isEmpty())
         {
             if (consistencyLevel == ConsistencyLevel.ANY)
-                return Collections.singleton(FBUtilities.getBroadcastAddress());
+                return Collections.singleton(FBUtilities.getBroadcastAddressAndPort());
 
             throw new UnavailableException(ConsistencyLevel.ONE, 1, 0);
         }
@@ -1246,7 +1243,7 @@ public class StorageProxy implements StorageProxyMBean
      * @throws OverloadedException if the hints cannot be written/enqueued
      */
     public static void sendToHintedEndpoints(final Mutation mutation,
-                                             Iterable<InetAddress> targets,
+                                             Iterable<InetAddressAndPort> targets,
                                              AbstractWriteResponseHandler<IMutation> responseHandler,
                                              String localDataCenter,
                                              Stage stage)
@@ -1255,18 +1252,18 @@ public class StorageProxy implements StorageProxyMBean
         int targetsSize = Iterables.size(targets);
 
         // this dc replicas:
-        Collection<InetAddress> localDc = null;
+        Collection<InetAddressAndPort> localDc = null;
         // extra-datacenter replicas, grouped by dc
-        Map<String, Collection<InetAddress>> dcGroups = null;
+        Map<String, Collection<InetAddressAndPort>> dcGroups = null;
         // only need to create a Message for non-local writes
         MessageOut<Mutation> message = null;
 
         boolean insertLocal = false;
-        ArrayList<InetAddress> endpointsToHint = null;
+        ArrayList<InetAddressAndPort> endpointsToHint = null;
 
-        List<InetAddress> backPressureHosts = null;
+        List<InetAddressAndPort> backPressureHosts = null;
 
-        for (InetAddress destination : targets)
+        for (InetAddressAndPort destination : targets)
         {
             checkHintOverload(destination);
 
@@ -1295,7 +1292,7 @@ public class StorageProxy implements StorageProxyMBean
                     }
                     else
                     {
-                        Collection<InetAddress> messages = (dcGroups != null) ? dcGroups.get(dc) : null;
+                        Collection<InetAddressAndPort> messages = (dcGroups != null) ? dcGroups.get(dc) : null;
                         if (messages == null)
                         {
                             messages = new ArrayList<>(3); // most DCs will have <= 3 replicas
@@ -1338,18 +1335,18 @@ public class StorageProxy implements StorageProxyMBean
 
         if (localDc != null)
         {
-            for (InetAddress destination : localDc)
+            for (InetAddressAndPort destination : localDc)
                 MessagingService.instance().sendRR(message, destination, responseHandler, true);
         }
         if (dcGroups != null)
         {
             // for each datacenter, send the message to one node to relay the write to other replicas
-            for (Collection<InetAddress> dcTargets : dcGroups.values())
+            for (Collection<InetAddressAndPort> dcTargets : dcGroups.values())
                 sendMessagesToNonlocalDC(message, dcTargets, responseHandler);
         }
     }
 
-    private static void checkHintOverload(InetAddress destination)
+    private static void checkHintOverload(InetAddressAndPort destination)
     {
         // avoid OOMing due to excess hints.  we need to do this check even for "live" nodes, since we can
         // still generate hints for those if it's overloaded or simply dead but not yet known-to-be-dead.
@@ -1366,39 +1363,31 @@ public class StorageProxy implements StorageProxyMBean
     }
 
     private static void sendMessagesToNonlocalDC(MessageOut<? extends IMutation> message,
-                                                 Collection<InetAddress> targets,
+                                                 Collection<InetAddressAndPort> targets,
                                                  AbstractWriteResponseHandler<IMutation> handler)
     {
-        Iterator<InetAddress> iter = targets.iterator();
-        InetAddress target = iter.next();
+        Iterator<InetAddressAndPort> iter = targets.iterator();
+        int[] messageIds = new int[targets.size()];
+        InetAddressAndPort target = iter.next();
 
+        int idIdx = 0;
         // Add the other destinations of the same message as a FORWARD_HEADER entry
-        try(DataOutputBuffer out = new DataOutputBuffer())
-        {
-            out.writeInt(targets.size() - 1);
-            while (iter.hasNext())
-            {
-                InetAddress destination = iter.next();
-                CompactEndpointSerializationHelper.serialize(destination, out);
-                int id = MessagingService.instance().addCallback(handler,
-                                                                 message,
-                                                                 destination,
-                                                                 message.getTimeout(),
-                                                                 handler.consistencyLevel,
-                                                                 true);
-                out.writeInt(id);
-                logger.trace("Adding FWD message to {}@{}", id, destination);
-            }
-            message = message.withParameter(Mutation.FORWARD_TO, out.getData());
-            // send the combined message + forward headers
-            int id = MessagingService.instance().sendRR(message, target, handler, true);
-            logger.trace("Sending message to {}@{}", id, target);
-        }
-        catch (IOException e)
-        {
-            // DataOutputBuffer is in-memory, doesn't throw IOException
-            throw new AssertionError(e);
+        while (iter.hasNext())
+        {
+            InetAddressAndPort destination = iter.next();
+            int id = MessagingService.instance().addCallback(handler,
+                                                             message,
+                                                             destination,
+                                                             message.getTimeout(),
+                                                             handler.consistencyLevel,
+                                                             true);
+            messageIds[idIdx++] = id;
+            logger.trace("Adding FWD message to {}@{}", id, destination);
         }
+        message = message.withParameter(ParameterType.FORWARD_TO.FORWARD_TO, new ForwardToContainer(targets, messageIds));
+        // send the combined message + forward headers
+        int id = MessagingService.instance().sendRR(message, target, handler, true);
+        logger.trace("Sending message to {}@{}", id, target);
     }
 
     private static void performLocally(Stage stage, final Runnable runnable)
@@ -1440,7 +1429,7 @@ public class StorageProxy implements StorageProxyMBean
                 {
                     if (!(ex instanceof WriteTimeoutException))
                         logger.error("Failed to apply mutation locally : ", ex);
-                    handler.onFailure(FBUtilities.getBroadcastAddress(), RequestFailureReason.UNKNOWN);
+                    handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN);
                 }
             }
 
@@ -1468,9 +1457,9 @@ public class StorageProxy implements StorageProxyMBean
      */
     public static AbstractWriteResponseHandler<IMutation> mutateCounter(CounterMutation cm, String localDataCenter, long queryStartNanoTime) throws UnavailableException, OverloadedException
     {
-        InetAddress endpoint = findSuitableEndpoint(cm.getKeyspaceName(), cm.key(), localDataCenter, cm.consistency());
+        InetAddressAndPort endpoint = findSuitableEndpoint(cm.getKeyspaceName(), cm.key(), localDataCenter, cm.consistency());
 
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
         {
             return applyCounterMutationOnCoordinator(cm, localDataCenter, queryStartNanoTime);
         }
@@ -1480,8 +1469,8 @@ public class StorageProxy implements StorageProxyMBean
             String keyspaceName = cm.getKeyspaceName();
             AbstractReplicationStrategy rs = Keyspace.open(keyspaceName).getReplicationStrategy();
             Token tk = cm.key().getToken();
-            List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
-            Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
+            List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
+            Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
 
             rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, cm.consistency(), null, WriteType.COUNTER, queryStartNanoTime).assureSufficientLiveNodes();
 
@@ -1504,11 +1493,11 @@ public class StorageProxy implements StorageProxyMBean
      * is unclear we want to mix those latencies with read latencies, so this
      * may be a bit involved.
      */
-    private static InetAddress findSuitableEndpoint(String keyspaceName, DecoratedKey key, String localDataCenter, ConsistencyLevel cl) throws UnavailableException
+    private static InetAddressAndPort findSuitableEndpoint(String keyspaceName, DecoratedKey key, String localDataCenter, ConsistencyLevel cl) throws UnavailableException
     {
         Keyspace keyspace = Keyspace.open(keyspaceName);
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        List<InetAddress> endpoints = new ArrayList<>();
+        List<InetAddressAndPort> endpoints = new ArrayList<>();
         StorageService.instance.getLiveNaturalEndpoints(keyspace, key, endpoints);
 
         // CASSANDRA-13043: filter out those endpoints not accepting clients yet, maybe because still bootstrapping
@@ -1518,9 +1507,9 @@ public class StorageProxy implements StorageProxyMBean
         if (endpoints.isEmpty())
             throw new UnavailableException(cl, cl.blockFor(keyspace), 0);
 
-        List<InetAddress> localEndpoints = new ArrayList<>(endpoints.size());
+        List<InetAddressAndPort> localEndpoints = new ArrayList<>(endpoints.size());
 
-        for (InetAddress endpoint : endpoints)
+        for (InetAddressAndPort endpoint : endpoints)
             if (snitch.getDatacenter(endpoint).equals(localDataCenter))
                 localEndpoints.add(endpoint);
 
@@ -1531,7 +1520,7 @@ public class StorageProxy implements StorageProxyMBean
                 throw new UnavailableException(cl, cl.blockFor(keyspace), 0);
 
             // No endpoint in local DC, pick the closest endpoint according to the snitch
-            snitch.sortByProximity(FBUtilities.getBroadcastAddress(), endpoints);
+            snitch.sortByProximity(FBUtilities.getBroadcastAddressAndPort(), endpoints);
             return endpoints.get(0);
         }
 
@@ -1555,7 +1544,7 @@ public class StorageProxy implements StorageProxyMBean
     }
 
     private static Runnable counterWriteTask(final IMutation mutation,
-                                             final Iterable<InetAddress> targets,
+                                             final Iterable<InetAddressAndPort> targets,
                                              final AbstractWriteResponseHandler<IMutation> responseHandler,
                                              final String localDataCenter)
     {
@@ -1569,8 +1558,8 @@ public class StorageProxy implements StorageProxyMBean
                 Mutation result = ((CounterMutation) mutation).applyCounterMutation();
                 responseHandler.response(null);
 
-                Set<InetAddress> remotes = Sets.difference(ImmutableSet.copyOf(targets),
-                                                           ImmutableSet.of(FBUtilities.getBroadcastAddress()));
+                Set<InetAddressAndPort> remotes = Sets.difference(ImmutableSet.copyOf(targets),
+                                                                  ImmutableSet.of(FBUtilities.getBroadcastAddressAndPort()));
                 if (!remotes.isEmpty())
                     sendToHintedEndpoints(result, remotes, responseHandler, localDataCenter, Stage.COUNTER_MUTATION);
             }
@@ -1640,8 +1629,8 @@ public class StorageProxy implements StorageProxyMBean
         try
         {
             // make sure any in-progress paxos writes are done (i.e., committed to a majority of replicas), before performing a quorum read
-            Pair<List<InetAddress>, Integer> p = getPaxosParticipants(metadata, key, consistencyLevel);
-            List<InetAddress> liveEndpoints = p.left;
+            Pair<List<InetAddressAndPort>, Integer> p = getPaxosParticipants(metadata, key, consistencyLevel);
+            List<InetAddressAndPort> liveEndpoints = p.left;
             int requiredParticipants = p.right;
 
             // does the work of applying in-progress writes; throws UAE or timeout if it can't
@@ -1844,7 +1833,7 @@ public class StorageProxy implements StorageProxyMBean
                                                  executor.handler.endpoints,
                                                  queryStartNanoTime);
 
-                for (InetAddress endpoint : executor.getContactedReplicas())
+                for (InetAddressAndPort endpoint : executor.getContactedReplicas())
                 {
                     Tracing.trace("Enqueuing full data read to {}", endpoint);
                     MessagingService.instance().sendRRWithFailure(command.createMessage(), endpoint, repairHandler);
@@ -1920,47 +1909,47 @@ public class StorageProxy implements StorageProxyMBean
                 else
                 {
                     MessagingService.instance().incrementDroppedMessages(verb, System.currentTimeMillis() - constructionTime);
-                    handler.onFailure(FBUtilities.getBroadcastAddress(), RequestFailureReason.UNKNOWN);
+                    handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN);
                 }
 
-                MessagingService.instance().addLatency(FBUtilities.getBroadcastAddress(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+                MessagingService.instance().addLatency(FBUtilities.getBroadcastAddressAndPort(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
             }
             catch (Throwable t)
             {
                 if (t instanceof TombstoneOverwhelmingException)
                 {
-                    handler.onFailure(FBUtilities.getBroadcastAddress(), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
+                    handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
                     logger.error(t.getMessage());
                 }
                 else
                 {
-                    handler.onFailure(FBUtilities.getBroadcastAddress(), RequestFailureReason.UNKNOWN);
+                    handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN);
                     throw t;
                 }
             }
         }
     }
 
-    public static List<InetAddress> getLiveSortedEndpoints(Keyspace keyspace, ByteBuffer key)
+    public static List<InetAddressAndPort> getLiveSortedEndpoints(Keyspace keyspace, ByteBuffer key)
     {
         return getLiveSortedEndpoints(keyspace, StorageService.instance.getTokenMetadata().decorateKey(key));
     }
 
-    public static List<InetAddress> getLiveSortedEndpoints(Keyspace keyspace, RingPosition pos)
+    public static List<InetAddressAndPort> getLiveSortedEndpoints(Keyspace keyspace, RingPosition pos)
     {
-        List<InetAddress> liveEndpoints = StorageService.instance.getLiveNaturalEndpoints(keyspace, pos);
-        DatabaseDescriptor.getEndpointSnitch().sortByProximity(FBUtilities.getBroadcastAddress(), liveEndpoints);
+        List<InetAddressAndPort> liveEndpoints = StorageService.instance.getLiveNaturalEndpoints(keyspace, pos);
+        DatabaseDescriptor.getEndpointSnitch().sortByProximity(FBUtilities.getBroadcastAddressAndPort(), liveEndpoints);
         return liveEndpoints;
     }
 
-    private static List<InetAddress> intersection(List<InetAddress> l1, List<InetAddress> l2)
+    private static List<InetAddressAndPort> intersection(List<InetAddressAndPort> l1, List<InetAddressAndPort> l2)
     {
         // Note: we don't use Guava Sets.intersection() for 3 reasons:
         //   1) retainAll would be inefficient if l1 and l2 are large but in practice both are the replicas for a range and
         //   so will be very small (< RF). In that case, retainAll is in fact more efficient.
         //   2) we do ultimately need a list so converting everything to sets don't make sense
         //   3) l1 and l2 are sorted by proximity. The use of retainAll  maintain that sorting in the result, while using sets wouldn't.
-        List<InetAddress> inter = new ArrayList<InetAddress>(l1);
+        List<InetAddressAndPort> inter = new ArrayList<>(l1);
         inter.retainAll(l2);
         return inter;
     }
@@ -1986,10 +1975,10 @@ public class StorageProxy implements StorageProxyMBean
     private static class RangeForQuery
     {
         public final AbstractBounds<PartitionPosition> range;
-        public final List<InetAddress> liveEndpoints;
-        public final List<InetAddress> filteredEndpoints;
+        public final List<InetAddressAndPort> liveEndpoints;
+        public final List<InetAddressAndPort> filteredEndpoints;
 
-        public RangeForQuery(AbstractBounds<PartitionPosition> range, List<InetAddress> liveEndpoints, List<InetAddress> filteredEndpoints)
+        public RangeForQuery(AbstractBounds<PartitionPosition> range, List<InetAddressAndPort> liveEndpoints, List<InetAddressAndPort> filteredEndpoints)
         {
             this.range = range;
             this.liveEndpoints = liveEndpoints;
@@ -2027,7 +2016,7 @@ public class StorageProxy implements StorageProxyMBean
                 return endOfData();
 
             AbstractBounds<PartitionPosition> range = ranges.next();
-            List<InetAddress> liveEndpoints = getLiveSortedEndpoints(keyspace, range.right);
+            List<InetAddressAndPort> liveEndpoints = getLiveSortedEndpoints(keyspace, range.right);
             return new RangeForQuery(range,
                                      liveEndpoints,
                                      consistency.filterForQuery(keyspace, liveEndpoints));
@@ -2069,13 +2058,13 @@ public class StorageProxy implements StorageProxyMBean
 
                 RangeForQuery next = ranges.peek();
 
-                List<InetAddress> merged = intersection(current.liveEndpoints, next.liveEndpoints);
+                List<InetAddressAndPort> merged = intersection(current.liveEndpoints, next.liveEndpoints);
 
                 // Check if there is enough endpoint for the merge to be possible.
                 if (!consistency.isSufficientLiveNodes(keyspace, merged))
                     break;
 
-                List<InetAddress> filteredMerged = consistency.filterForQuery(keyspace, merged);
+                List<InetAddressAndPort> filteredMerged = consistency.filterForQuery(keyspace, merged);
 
                 // Estimate whether merging will be a win or not
                 if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(filteredMerged, current.filteredEndpoints, next.filteredEndpoints))
@@ -2237,7 +2226,7 @@ public class StorageProxy implements StorageProxyMBean
 
             int blockFor = consistency.blockFor(keyspace);
             int minResponses = Math.min(toQuery.filteredEndpoints.size(), blockFor);
-            List<InetAddress> minimalEndpoints = toQuery.filteredEndpoints.subList(0, minResponses);
+            List<InetAddressAndPort> minimalEndpoints = toQuery.filteredEndpoints.subList(0, minResponses);
             ReadCallback handler = new ReadCallback(resolver, consistency, rangeCommand, minimalEndpoints, queryStartNanoTime);
 
             handler.assureSufficientLiveNodes();
@@ -2248,7 +2237,7 @@ public class StorageProxy implements StorageProxyMBean
             }
             else
             {
-                for (InetAddress endpoint : toQuery.filteredEndpoints)
+                for (InetAddressAndPort endpoint : toQuery.filteredEndpoints)
                 {
                     Tracing.trace("Enqueuing request to {}", endpoint);
                     MessagingService.instance().sendRRWithFailure(rangeCommand.createMessage(), endpoint, handler);
@@ -2320,7 +2309,12 @@ public class StorageProxy implements StorageProxyMBean
 
     public Map<String, List<String>> getSchemaVersions()
     {
-        return describeSchemaVersions();
+        return describeSchemaVersions(false);
+    }
+
+    public Map<String, List<String>> getSchemaVersionsWithPort()
+    {
+        return describeSchemaVersions(true);
     }
 
     /**
@@ -2328,11 +2322,11 @@ public class StorageProxy implements StorageProxyMBean
      * migration id. This is useful for determining if a schema change has propagated through the cluster. Disagreement
      * is assumed if any node fails to respond.
      */
-    public static Map<String, List<String>> describeSchemaVersions()
+    public static Map<String, List<String>> describeSchemaVersions(boolean withPort)
     {
         final String myVersion = Schema.instance.getVersion().toString();
-        final Map<InetAddress, UUID> versions = new ConcurrentHashMap<InetAddress, UUID>();
-        final Set<InetAddress> liveHosts = Gossiper.instance.getLiveMembers();
+        final Map<InetAddressAndPort, UUID> versions = new ConcurrentHashMap<>();
+        final Set<InetAddressAndPort> liveHosts = Gossiper.instance.getLiveMembers();
         final CountDownLatch latch = new CountDownLatch(liveHosts.size());
 
         IAsyncCallback<UUID> cb = new IAsyncCallback<UUID>()
@@ -2351,7 +2345,7 @@ public class StorageProxy implements StorageProxyMBean
         };
         // an empty message acts as a request to the SchemaVersionVerbHandler.
         MessageOut message = new MessageOut(MessagingService.Verb.SCHEMA_CHECK);
-        for (InetAddress endpoint : liveHosts)
+        for (InetAddressAndPort endpoint : liveHosts)
             MessagingService.instance().sendRR(message, endpoint, cb);
 
         try
@@ -2366,8 +2360,8 @@ public class StorageProxy implements StorageProxyMBean
 
         // maps versions to hosts that are on that version.
         Map<String, List<String>> results = new HashMap<String, List<String>>();
-        Iterable<InetAddress> allHosts = Iterables.concat(Gossiper.instance.getLiveMembers(), Gossiper.instance.getUnreachableMembers());
-        for (InetAddress host : allHosts)
+        Iterable<InetAddressAndPort> allHosts = Iterables.concat(Gossiper.instance.getLiveMembers(), Gossiper.instance.getUnreachableMembers());
+        for (InetAddressAndPort host : allHosts)
         {
             UUID version = versions.get(host);
             String stringVersion = version == null ? UNREACHABLE : version.toString();
@@ -2377,7 +2371,7 @@ public class StorageProxy implements StorageProxyMBean
                 hosts = new ArrayList<String>();
                 results.put(stringVersion, hosts);
             }
-            hosts.add(host.getHostAddress());
+            hosts.add(host.getHostAddress(withPort));
         }
 
         // we're done: the results map is ready to return to the client.  the rest is just debug logging:
@@ -2485,7 +2479,7 @@ public class StorageProxy implements StorageProxyMBean
         DatabaseDescriptor.setMaxHintWindow(ms);
     }
 
-    public static boolean shouldHint(InetAddress ep)
+    public static boolean shouldHint(InetAddressAndPort ep)
     {
         if (DatabaseDescriptor.hintedHandoffEnabled())
         {
@@ -2534,7 +2528,7 @@ public class StorageProxy implements StorageProxyMBean
             throw new UnavailableException(ConsistencyLevel.ALL, liveMembers + Gossiper.instance.getUnreachableMembers().size(), liveMembers);
         }
 
-        Set<InetAddress> allEndpoints = StorageService.instance.getLiveRingMembers(true);
+        Set<InetAddressAndPort> allEndpoints = StorageService.instance.getLiveRingMembers(true);
 
         int blockFor = allEndpoints.size();
         final TruncateResponseHandler responseHandler = new TruncateResponseHandler(blockFor);
@@ -2543,7 +2537,7 @@ public class StorageProxy implements StorageProxyMBean
         Tracing.trace("Enqueuing truncate messages to hosts {}", allEndpoints);
         final Truncation truncation = new Truncation(keyspace, cfname);
         MessageOut<Truncation> message = truncation.createMessage();
-        for (InetAddress endpoint : allEndpoints)
+        for (InetAddressAndPort endpoint : allEndpoints)
             MessagingService.instance().sendRR(message, endpoint, responseHandler);
 
         // Wait for all
@@ -2570,7 +2564,7 @@ public class StorageProxy implements StorageProxyMBean
     public interface WritePerformer
     {
         public void apply(IMutation mutation,
-                          Iterable<InetAddress> targets,
+                          Iterable<InetAddressAndPort> targets,
                           AbstractWriteResponseHandler<IMutation> responseHandler,
                           String localDataCenter,
                           ConsistencyLevel consistencyLevel) throws OverloadedException;
@@ -2658,7 +2652,7 @@ public class StorageProxy implements StorageProxyMBean
             {
                 if (MessagingService.DROPPABLE_VERBS.contains(verb))
                     MessagingService.instance().incrementDroppedMutations(mutationOpt, timeTaken);
-                HintRunnable runnable = new HintRunnable(Collections.singleton(FBUtilities.getBroadcastAddress()))
+                HintRunnable runnable = new HintRunnable(Collections.singleton(FBUtilities.getBroadcastAddressAndPort()))
                 {
                     protected void runMayThrow() throws Exception
                     {
@@ -2689,9 +2683,9 @@ public class StorageProxy implements StorageProxyMBean
      */
     private abstract static class HintRunnable implements Runnable
     {
-        public final Collection<InetAddress> targets;
+        public final Collection<InetAddressAndPort> targets;
 
-        protected HintRunnable(Collection<InetAddress> targets)
+        protected HintRunnable(Collection<InetAddressAndPort> targets)
         {
             this.targets = targets;
         }
@@ -2709,7 +2703,7 @@ public class StorageProxy implements StorageProxyMBean
             finally
             {
                 StorageMetrics.totalHintsInProgress.dec(targets.size());
-                for (InetAddress target : targets)
+                for (InetAddressAndPort target : targets)
                     getHintsInProgressFor(target).decrementAndGet();
             }
         }
@@ -2743,7 +2737,7 @@ public class StorageProxy implements StorageProxyMBean
             logger.warn("Some hints were not written before shutdown.  This is not supposed to happen.  You should (a) run repair, and (b) file a bug report");
     }
 
-    private static AtomicInteger getHintsInProgressFor(InetAddress destination)
+    private static AtomicInteger getHintsInProgressFor(InetAddressAndPort destination)
     {
         try
         {
@@ -2755,22 +2749,22 @@ public class StorageProxy implements StorageProxyMBean
         }
     }
 
-    public static Future<Void> submitHint(Mutation mutation, InetAddress target, AbstractWriteResponseHandler<IMutation> responseHandler)
+    public static Future<Void> submitHint(Mutation mutation, InetAddressAndPort target, AbstractWriteResponseHandler<IMutation> responseHandler)
     {
         return submitHint(mutation, Collections.singleton(target), responseHandler);
     }
 
     public static Future<Void> submitHint(Mutation mutation,
-                                          Collection<InetAddress> targets,
+                                          Collection<InetAddressAndPort> targets,
                                           AbstractWriteResponseHandler<IMutation> responseHandler)
     {
         HintRunnable runnable = new HintRunnable(targets)
         {
             public void runMayThrow()
             {
-                Set<InetAddress> validTargets = new HashSet<>(targets.size());
+                Set<InetAddressAndPort> validTargets = new HashSet<>(targets.size());
                 Set<UUID> hostIds = new HashSet<>(targets.size());
-                for (InetAddress target : targets)
+                for (InetAddressAndPort target : targets)
                 {
                     UUID hostId = StorageService.instance.getHostIdForEndpoint(target);
                     if (hostId != null)
@@ -2796,7 +2790,7 @@ public class StorageProxy implements StorageProxyMBean
     private static Future<Void> submitHint(HintRunnable runnable)
     {
         StorageMetrics.totalHintsInProgress.inc(runnable.targets.size());
-        for (InetAddress target : runnable.targets)
+        for (InetAddressAndPort target : runnable.targets)
             getHintsInProgressFor(target).incrementAndGet();
         return (Future<Void>) StageManager.getStage(Stage.MUTATION).submit(runnable);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/StorageProxyMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxyMBean.java b/src/java/org/apache/cassandra/service/StorageProxyMBean.java
index 173d43f..76a6617 100644
--- a/src/java/org/apache/cassandra/service/StorageProxyMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageProxyMBean.java
@@ -65,7 +65,8 @@ public interface StorageProxyMBean
     public void setOtcBacklogExpirationInterval(int intervalInMillis);
 
     /** Returns each live node's schema version */
-    public Map<String, List<String>> getSchemaVersions();
+    @Deprecated public Map<String, List<String>> getSchemaVersions();
+    public Map<String, List<String>> getSchemaVersionsWithPort();
 
     public int getNumberOfTables();
 


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


[15/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/PendingRangeMaps.java b/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
index cfeccc4..92307a3 100644
--- a/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
+++ b/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
@@ -26,10 +26,9 @@ import org.apache.cassandra.dht.Token;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.InetAddress;
 import java.util.*;
 
-public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<InetAddress>>>
+public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<InetAddressAndPort>>>
 {
     private static final Logger logger = LoggerFactory.getLogger(PendingRangeMaps.class);
 
@@ -39,7 +38,7 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
      * First two are for non-wrap-around ranges, and the last two are for wrap-around ranges.
      */
     // ascendingMap will sort the ranges by the ascending order of right token
-    final NavigableMap<Range<Token>, List<InetAddress>> ascendingMap;
+    final NavigableMap<Range<Token>, List<InetAddressAndPort>> ascendingMap;
     /**
      * sorting end ascending, if ends are same, sorting begin descending, so that token (end, end) will
      * come before (begin, end] with the same end, and (begin, end) will be selected in the tailMap.
@@ -58,7 +57,7 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
         };
 
     // ascendingMap will sort the ranges by the descending order of left token
-    final NavigableMap<Range<Token>, List<InetAddress>> descendingMap;
+    final NavigableMap<Range<Token>, List<InetAddressAndPort>> descendingMap;
     /**
      * sorting begin descending, if begins are same, sorting end descending, so that token (begin, begin) will
      * come after (begin, end] with the same begin, and (begin, end) won't be selected in the tailMap.
@@ -78,7 +77,7 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
         };
 
     // these two maps are for warp around ranges.
-    final NavigableMap<Range<Token>, List<InetAddress>> ascendingMapForWrapAround;
+    final NavigableMap<Range<Token>, List<InetAddressAndPort>> ascendingMapForWrapAround;
     /**
      * for wrap around range (begin, end], which begin > end.
      * Sorting end ascending, if ends are same, sorting begin ascending,
@@ -98,7 +97,7 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
         }
     };
 
-    final NavigableMap<Range<Token>, List<InetAddress>> descendingMapForWrapAround;
+    final NavigableMap<Range<Token>, List<InetAddressAndPort>> descendingMapForWrapAround;
     /**
      * for wrap around ranges, which begin > end.
      * Sorting end ascending, so that token (begin, begin) will come after (begin, end] with the same begin,
@@ -118,28 +117,28 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
 
     public PendingRangeMaps()
     {
-        this.ascendingMap = new TreeMap<Range<Token>, List<InetAddress>>(ascendingComparator);
-        this.descendingMap = new TreeMap<Range<Token>, List<InetAddress>>(descendingComparator);
-        this.ascendingMapForWrapAround = new TreeMap<Range<Token>, List<InetAddress>>(ascendingComparatorForWrapAround);
-        this.descendingMapForWrapAround = new TreeMap<Range<Token>, List<InetAddress>>(descendingComparatorForWrapAround);
+        this.ascendingMap = new TreeMap<Range<Token>, List<InetAddressAndPort>>(ascendingComparator);
+        this.descendingMap = new TreeMap<Range<Token>, List<InetAddressAndPort>>(descendingComparator);
+        this.ascendingMapForWrapAround = new TreeMap<Range<Token>, List<InetAddressAndPort>>(ascendingComparatorForWrapAround);
+        this.descendingMapForWrapAround = new TreeMap<Range<Token>, List<InetAddressAndPort>>(descendingComparatorForWrapAround);
     }
 
     static final void addToMap(Range<Token> range,
-                               InetAddress address,
-                               NavigableMap<Range<Token>, List<InetAddress>> ascendingMap,
-                               NavigableMap<Range<Token>, List<InetAddress>> descendingMap)
+                               InetAddressAndPort address,
+                               NavigableMap<Range<Token>, List<InetAddressAndPort>> ascendingMap,
+                               NavigableMap<Range<Token>, List<InetAddressAndPort>> descendingMap)
     {
-        List<InetAddress> addresses = ascendingMap.get(range);
+        List<InetAddressAndPort> addresses = ascendingMap.get(range);
         if (addresses == null)
         {
-            addresses = new ArrayList<InetAddress>(1);
+            addresses = new ArrayList<>(1);
             ascendingMap.put(range, addresses);
             descendingMap.put(range, addresses);
         }
         addresses.add(address);
     }
 
-    public void addPendingRange(Range<Token> range, InetAddress address)
+    public void addPendingRange(Range<Token> range, InetAddressAndPort address)
     {
         if (Range.isWrapAround(range.left, range.right))
         {
@@ -151,14 +150,14 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
         }
     }
 
-    static final void addIntersections(Set<InetAddress> endpointsToAdd,
-                                       NavigableMap<Range<Token>, List<InetAddress>> smallerMap,
-                                       NavigableMap<Range<Token>, List<InetAddress>> biggerMap)
+    static final void addIntersections(Set<InetAddressAndPort> endpointsToAdd,
+                                       NavigableMap<Range<Token>, List<InetAddressAndPort>> smallerMap,
+                                       NavigableMap<Range<Token>, List<InetAddressAndPort>> biggerMap)
     {
         // find the intersection of two sets
         for (Range<Token> range : smallerMap.keySet())
         {
-            List<InetAddress> addresses = biggerMap.get(range);
+            List<InetAddressAndPort> addresses = biggerMap.get(range);
             if (addresses != null)
             {
                 endpointsToAdd.addAll(addresses);
@@ -166,15 +165,15 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
         }
     }
 
-    public Collection<InetAddress> pendingEndpointsFor(Token token)
+    public Collection<InetAddressAndPort> pendingEndpointsFor(Token token)
     {
-        Set<InetAddress> endpoints = new HashSet<>();
+        Set<InetAddressAndPort> endpoints = new HashSet<>();
 
         Range searchRange = new Range(token, token);
 
         // search for non-wrap-around maps
-        NavigableMap<Range<Token>, List<InetAddress>> ascendingTailMap = ascendingMap.tailMap(searchRange, true);
-        NavigableMap<Range<Token>, List<InetAddress>> descendingTailMap = descendingMap.tailMap(searchRange, false);
+        NavigableMap<Range<Token>, List<InetAddressAndPort>> ascendingTailMap = ascendingMap.tailMap(searchRange, true);
+        NavigableMap<Range<Token>, List<InetAddressAndPort>> descendingTailMap = descendingMap.tailMap(searchRange, false);
 
         // add intersections of two maps
         if (ascendingTailMap.size() < descendingTailMap.size())
@@ -191,11 +190,11 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
         descendingTailMap = descendingMapForWrapAround.tailMap(searchRange, false);
 
         // add them since they are all necessary.
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : ascendingTailMap.entrySet())
+        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : ascendingTailMap.entrySet())
         {
             endpoints.addAll(entry.getValue());
         }
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : descendingTailMap.entrySet())
+        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : descendingTailMap.entrySet())
         {
             endpoints.addAll(entry.getValue());
         }
@@ -207,11 +206,11 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
     {
         StringBuilder sb = new StringBuilder();
 
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : this)
+        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : this)
         {
             Range<Token> range = entry.getKey();
 
-            for (InetAddress address : entry.getValue())
+            for (InetAddressAndPort address : entry.getValue())
             {
                 sb.append(address).append(':').append(range);
                 sb.append(System.getProperty("line.separator"));
@@ -222,7 +221,7 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
     }
 
     @Override
-    public Iterator<Map.Entry<Range<Token>, List<InetAddress>>> iterator()
+    public Iterator<Map.Entry<Range<Token>, List<InetAddressAndPort>>> iterator()
     {
         return Iterators.concat(ascendingMap.entrySet().iterator(), ascendingMapForWrapAround.entrySet().iterator());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 2908976..3a9b161 100644
--- a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.locator;
 
 import java.io.InputStream;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -55,7 +54,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
     public static final String SNITCH_PROPERTIES_FILENAME = "cassandra-topology.properties";
     private static final int DEFAULT_REFRESH_PERIOD_IN_SECONDS = 5;
 
-    private static volatile Map<InetAddress, String[]> endpointMap;
+    private static volatile Map<InetAddressAndPort, String[]> endpointMap;
     private static volatile String[] defaultDCRack;
 
     private volatile boolean gossipStarted;
@@ -93,7 +92,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
      * @param endpoint endpoint to process
      * @return a array of string with the first index being the data center and the second being the rack
      */
-    public static String[] getEndpointInfo(InetAddress endpoint)
+    public static String[] getEndpointInfo(InetAddressAndPort endpoint)
     {
         String[] rawEndpointInfo = getRawEndpointInfo(endpoint);
         if (rawEndpointInfo == null)
@@ -101,7 +100,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
         return rawEndpointInfo;
     }
 
-    private static String[] getRawEndpointInfo(InetAddress endpoint)
+    private static String[] getRawEndpointInfo(InetAddressAndPort endpoint)
     {
         String[] value = endpointMap.get(endpoint);
         if (value == null)
@@ -118,7 +117,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
      * @param endpoint the endpoint to process
      * @return string of data center
      */
-    public String getDatacenter(InetAddress endpoint)
+    public String getDatacenter(InetAddressAndPort endpoint)
     {
         String[] info = getEndpointInfo(endpoint);
         assert info != null : "No location defined for endpoint " + endpoint;
@@ -131,7 +130,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
      * @param endpoint the endpoint to process
      * @return string of rack
      */
-    public String getRack(InetAddress endpoint)
+    public String getRack(InetAddressAndPort endpoint)
     {
         String[] info = getEndpointInfo(endpoint);
         assert info != null : "No location defined for endpoint " + endpoint;
@@ -140,7 +139,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
 
     public void reloadConfiguration(boolean isUpdate) throws ConfigurationException
     {
-        HashMap<InetAddress, String[]> reloadedMap = new HashMap<>();
+        HashMap<InetAddressAndPort, String[]> reloadedMap = new HashMap<>();
         String[] reloadedDefaultDCRack = null;
 
         Properties properties = new Properties();
@@ -168,11 +167,11 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
             }
             else
             {
-                InetAddress host;
+                InetAddressAndPort host;
                 String hostString = StringUtils.remove(key, '/');
                 try
                 {
-                    host = InetAddress.getByName(hostString);
+                    host = InetAddressAndPort.getByName(hostString);
                 }
                 catch (UnknownHostException e)
                 {
@@ -186,7 +185,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
                 reloadedMap.put(host, token);
             }
         }
-        InetAddress broadcastAddress = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort broadcastAddress = FBUtilities.getBroadcastAddressAndPort();
         String[] localInfo = reloadedMap.get(broadcastAddress);
         if (reloadedDefaultDCRack == null && localInfo == null)
             throw new ConfigurationException(String.format("Snitch definitions at %s do not define a location for " +
@@ -194,7 +193,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
                                                            SNITCH_PROPERTIES_FILENAME, broadcastAddress));
         // internode messaging code converts our broadcast address to local,
         // make sure we can be found at that as well.
-        InetAddress localAddress = FBUtilities.getLocalAddress();
+        InetAddressAndPort localAddress = FBUtilities.getLocalAddressAndPort();
         if (!localAddress.equals(broadcastAddress) && !reloadedMap.containsKey(localAddress))
             reloadedMap.put(localAddress, localInfo);
 
@@ -204,7 +203,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
         if (logger.isTraceEnabled())
         {
             StringBuilder sb = new StringBuilder();
-            for (Map.Entry<InetAddress, String[]> entry : reloadedMap.entrySet())
+            for (Map.Entry<InetAddressAndPort, String[]> entry : reloadedMap.entrySet())
                 sb.append(entry.getKey()).append(':').append(Arrays.toString(entry.getValue())).append(", ");
             logger.trace("Loaded network topology from property file: {}", StringUtils.removeEnd(sb.toString(), ", "));
         }
@@ -231,17 +230,17 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
      * @param reloadedDefaultDCRack - the default dc:rack or null if no default
      * @return true if we can continue updating (no live host had dc or rack updated)
      */
-    private static boolean livenessCheck(HashMap<InetAddress, String[]> reloadedMap, String[] reloadedDefaultDCRack)
+    private static boolean livenessCheck(HashMap<InetAddressAndPort, String[]> reloadedMap, String[] reloadedDefaultDCRack)
     {
         // If the default has changed we must check all live hosts but hopefully we will find a live
         // host quickly and interrupt the loop. Otherwise we only check the live hosts that were either
         // in the old set or in the new set
-        Set<InetAddress> hosts = Arrays.equals(defaultDCRack, reloadedDefaultDCRack)
+        Set<InetAddressAndPort> hosts = Arrays.equals(defaultDCRack, reloadedDefaultDCRack)
                                  ? Sets.intersection(StorageService.instance.getLiveRingMembers(), // same default
                                                      Sets.union(endpointMap.keySet(), reloadedMap.keySet()))
                                  : StorageService.instance.getLiveRingMembers(); // default updated
 
-        for (InetAddress host : hosts)
+        for (InetAddressAndPort host : hosts)
         {
             String[] origValue = endpointMap.containsKey(host) ? endpointMap.get(host) : defaultDCRack;
             String[] updateValue = reloadedMap.containsKey(host) ? reloadedMap.get(host) : reloadedDefaultDCRack;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/RackInferringSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/RackInferringSnitch.java b/src/java/org/apache/cassandra/locator/RackInferringSnitch.java
index a6ea1ab..6ae10cc 100644
--- a/src/java/org/apache/cassandra/locator/RackInferringSnitch.java
+++ b/src/java/org/apache/cassandra/locator/RackInferringSnitch.java
@@ -17,21 +17,19 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
-
 /**
  * A simple endpoint snitch implementation that assumes datacenter and rack information is encoded
  * in the 2nd and 3rd octets of the ip address, respectively.
  */
 public class RackInferringSnitch extends AbstractNetworkTopologySnitch
 {
-    public String getRack(InetAddress endpoint)
+    public String getRack(InetAddressAndPort endpoint)
     {
-        return Integer.toString(endpoint.getAddress()[2] & 0xFF, 10);
+        return Integer.toString(endpoint.address.getAddress()[2] & 0xFF, 10);
     }
 
-    public String getDatacenter(InetAddress endpoint)
+    public String getDatacenter(InetAddressAndPort endpoint)
     {
-        return Integer.toString(endpoint.getAddress()[1] & 0xFF, 10);
+        return Integer.toString(endpoint.address.getAddress()[1] & 0xFF, 10);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 0b344c9..5479010 100644
--- a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
+++ b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -49,11 +48,11 @@ public class ReconnectableSnitchHelper implements IEndpointStateChangeSubscriber
         this.preferLocal = preferLocal;
     }
 
-    private void reconnect(InetAddress publicAddress, VersionedValue localAddressValue)
+    private void reconnect(InetAddressAndPort publicAddress, VersionedValue localAddressValue)
     {
         try
         {
-            reconnect(publicAddress, InetAddress.getByName(localAddressValue.value), snitch, localDc);
+            reconnect(publicAddress, InetAddressAndPort.getByName(localAddressValue.value), snitch, localDc);
         }
         catch (UnknownHostException e)
         {
@@ -62,9 +61,9 @@ public class ReconnectableSnitchHelper implements IEndpointStateChangeSubscriber
     }
 
     @VisibleForTesting
-    static void reconnect(InetAddress publicAddress, InetAddress localAddress, IEndpointSnitch snitch, String localDc)
+    static void reconnect(InetAddressAndPort publicAddress, InetAddressAndPort localAddress, IEndpointSnitch snitch, String localDc)
     {
-        if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(publicAddress, MessagingService.instance().portFor(publicAddress)))
+        if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(publicAddress.address, MessagingService.instance().portFor(publicAddress)))
         {
             logger.debug("InternodeAuthenticator said don't reconnect to {} on {}", publicAddress, localAddress);
             return;
@@ -78,40 +77,65 @@ public class ReconnectableSnitchHelper implements IEndpointStateChangeSubscriber
         }
     }
 
-    public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue)
+    public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue)
     {
         // no-op
     }
 
-    public void onJoin(InetAddress endpoint, EndpointState epState)
+    public void onJoin(InetAddressAndPort endpoint, EndpointState epState)
     {
-        if (preferLocal && !Gossiper.instance.isDeadState(epState) && epState.getApplicationState(ApplicationState.INTERNAL_IP) != null)
-            reconnect(endpoint, epState.getApplicationState(ApplicationState.INTERNAL_IP));
+        if (preferLocal && !Gossiper.instance.isDeadState(epState))
+        {
+            VersionedValue address = epState.getApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT);
+            if (address == null)
+            {
+                address = epState.getApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT);
+            }
+            if (address != null)
+            {
+                reconnect(endpoint, address);
+            }
+        }
     }
 
-    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value)
+    //Skeptical this will always do the right thing all the time port wise. It will converge on the right thing
+    //eventually once INTERNAL_ADDRESS_AND_PORT is populated
+    public void onChange(InetAddressAndPort endpoint, ApplicationState state, VersionedValue value)
     {
-        if (preferLocal && state == ApplicationState.INTERNAL_IP && !Gossiper.instance.isDeadState(Gossiper.instance.getEndpointStateForEndpoint(endpoint)))
-            reconnect(endpoint, value);
+        if (preferLocal && !Gossiper.instance.isDeadState(Gossiper.instance.getEndpointStateForEndpoint(endpoint)))
+        {
+            if (state == ApplicationState.INTERNAL_ADDRESS_AND_PORT)
+            {
+                reconnect(endpoint, value);
+            }
+            else if (state == ApplicationState.INTERNAL_IP &&
+                     null == Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT))
+            {
+                //Only use INTERNAL_IP if INTERNAL_ADDRESS_AND_PORT is unavailable
+                reconnect(endpoint, value);
+            }
+        }
     }
 
-    public void onAlive(InetAddress endpoint, EndpointState state)
+    public void onAlive(InetAddressAndPort endpoint, EndpointState state)
     {
-        if (preferLocal && state.getApplicationState(ApplicationState.INTERNAL_IP) != null)
-            reconnect(endpoint, state.getApplicationState(ApplicationState.INTERNAL_IP));
+        VersionedValue internalIP = state.getApplicationState(ApplicationState.INTERNAL_IP);
+        VersionedValue internalIPAndPorts = state.getApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT);
+        if (preferLocal && internalIP != null)
+            reconnect(endpoint, internalIPAndPorts != null ? internalIPAndPorts : internalIP);
     }
 
-    public void onDead(InetAddress endpoint, EndpointState state)
+    public void onDead(InetAddressAndPort endpoint, EndpointState state)
     {
         // do nothing.
     }
 
-    public void onRemove(InetAddress endpoint)
+    public void onRemove(InetAddressAndPort endpoint)
     {
         // do nothing.
     }
 
-    public void onRestart(InetAddress endpoint, EndpointState state)
+    public void onRestart(InetAddressAndPort endpoint, EndpointState state)
     {
         // do nothing.
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/SeedProvider.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SeedProvider.java b/src/java/org/apache/cassandra/locator/SeedProvider.java
index a013fbb..7efa9e0 100644
--- a/src/java/org/apache/cassandra/locator/SeedProvider.java
+++ b/src/java/org/apache/cassandra/locator/SeedProvider.java
@@ -17,10 +17,9 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.List;
 
 public interface SeedProvider
 {
-    List<InetAddress> getSeeds();
+    List<InetAddressAndPort> getSeeds();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java b/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java
index 665261d..47401a0 100644
--- a/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java
+++ b/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -26,6 +25,7 @@ import java.util.Map;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,7 +35,7 @@ public class SimpleSeedProvider implements SeedProvider
 
     public SimpleSeedProvider(Map<String, String> args) {}
 
-    public List<InetAddress> getSeeds()
+    public List<InetAddressAndPort> getSeeds()
     {
         Config conf;
         try
@@ -47,12 +47,12 @@ public class SimpleSeedProvider implements SeedProvider
             throw new AssertionError(e);
         }
         String[] hosts = conf.seed_provider.parameters.get("seeds").split(",", -1);
-        List<InetAddress> seeds = new ArrayList<InetAddress>(hosts.length);
+        List<InetAddressAndPort> seeds = new ArrayList<>(hosts.length);
         for (String host : hosts)
         {
             try
             {
-                seeds.add(InetAddress.getByName(host.trim()));
+                seeds.add(InetAddressAndPort.getByName(host.trim()));
             }
             catch (UnknownHostException ex)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/SimpleSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SimpleSnitch.java b/src/java/org/apache/cassandra/locator/SimpleSnitch.java
index 27648c8..e31fc6b 100644
--- a/src/java/org/apache/cassandra/locator/SimpleSnitch.java
+++ b/src/java/org/apache/cassandra/locator/SimpleSnitch.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.List;
 
 /**
@@ -27,23 +26,23 @@ import java.util.List;
  */
 public class SimpleSnitch extends AbstractEndpointSnitch
 {
-    public String getRack(InetAddress endpoint)
+    public String getRack(InetAddressAndPort endpoint)
     {
         return "rack1";
     }
 
-    public String getDatacenter(InetAddress endpoint)
+    public String getDatacenter(InetAddressAndPort endpoint)
     {
         return "datacenter1";
     }
 
     @Override
-    public void sortByProximity(final InetAddress address, List<InetAddress> addresses)
+    public void sortByProximity(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
     {
         // Optimization to avoid walking the list
     }
 
-    public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+    public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
     {
         // Making all endpoints equal ensures we won't change the original ordering (since
         // Collections.sort is guaranteed to be stable)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/SimpleStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SimpleStrategy.java b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
index 9a5062b..545ad28 100644
--- a/src/java/org/apache/cassandra/locator/SimpleStrategy.java
+++ b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collection;
@@ -42,11 +41,11 @@ public class SimpleStrategy extends AbstractReplicationStrategy
         super(keyspaceName, tokenMetadata, snitch, configOptions);
     }
 
-    public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+    public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
     {
         int replicas = getReplicationFactor();
         ArrayList<Token> tokens = metadata.sortedTokens();
-        List<InetAddress> endpoints = new ArrayList<InetAddress>(replicas);
+        List<InetAddressAndPort> endpoints = new ArrayList<InetAddressAndPort>(replicas);
 
         if (tokens.isEmpty())
             return endpoints;
@@ -55,7 +54,7 @@ public class SimpleStrategy extends AbstractReplicationStrategy
         Iterator<Token> iter = TokenMetadata.ringIterator(tokens, token, false);
         while (endpoints.size() < replicas && iter.hasNext())
         {
-            InetAddress ep = metadata.getEndpoint(iter.next());
+            InetAddressAndPort ep = metadata.getEndpoint(iter.next());
             if (!endpoints.contains(ep))
                 endpoints.add(ep);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/TokenMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index 00f9536..e2c4628 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
@@ -52,12 +51,12 @@ public class TokenMetadata
      * Each Token is associated with exactly one Address, but each Address may have
      * multiple tokens.  Hence, the BiMultiValMap collection.
      */
-    private final BiMultiValMap<Token, InetAddress> tokenToEndpointMap;
+    private final BiMultiValMap<Token, InetAddressAndPort> tokenToEndpointMap;
 
     /** Maintains endpoint to host ID map of every node in the cluster */
-    private final BiMap<InetAddress, UUID> endpointToHostIdMap;
+    private final BiMap<InetAddressAndPort, UUID> endpointToHostIdMap;
 
-    // Prior to CASSANDRA-603, we just had <tt>Map<Range, InetAddress> pendingRanges<tt>,
+    // Prior to CASSANDRA-603, we just had <tt>Map<Range, InetAddressAndPort> pendingRanges<tt>,
     // which was added to when a node began bootstrap and removed from when it finished.
     //
     // This is inadequate when multiple changes are allowed simultaneously.  For example,
@@ -70,8 +69,8 @@ public class TokenMetadata
     //
     // So, we made two changes:
     //
-    // First, we changed pendingRanges to a <tt>Multimap<Range, InetAddress></tt> (now
-    // <tt>Map<String, Multimap<Range, InetAddress>></tt>, because replication strategy
+    // First, we changed pendingRanges to a <tt>Multimap<Range, InetAddressAndPort></tt> (now
+    // <tt>Map<String, Multimap<Range, InetAddressAndPort>></tt>, because replication strategy
     // and options are per-KeySpace).
     //
     // Second, we added the bootstrapTokens and leavingEndpoints collections, so we can
@@ -81,17 +80,17 @@ public class TokenMetadata
     // Finally, note that recording the tokens of joining nodes in bootstrapTokens also
     // means we can detect and reject the addition of multiple nodes at the same token
     // before one becomes part of the ring.
-    private final BiMultiValMap<Token, InetAddress> bootstrapTokens = new BiMultiValMap<>();
+    private final BiMultiValMap<Token, InetAddressAndPort> bootstrapTokens = new BiMultiValMap<>();
 
-    private final BiMap<InetAddress, InetAddress> replacementToOriginal = HashBiMap.create();
+    private final BiMap<InetAddressAndPort, InetAddressAndPort> replacementToOriginal = HashBiMap.create();
 
     // (don't need to record Token here since it's still part of tokenToEndpointMap until it's done leaving)
-    private final Set<InetAddress> leavingEndpoints = new HashSet<>();
+    private final Set<InetAddressAndPort> leavingEndpoints = new HashSet<>();
     // this is a cache of the calculation from {tokenToEndpointMap, bootstrapTokens, leavingEndpoints}
     private final ConcurrentMap<String, PendingRangeMaps> pendingRanges = new ConcurrentHashMap<String, PendingRangeMaps>();
 
     // nodes which are migrating to the new tokens in the ring
-    private final Set<Pair<Token, InetAddress>> movingEndpoints = new HashSet<>();
+    private final Set<Pair<Token, InetAddressAndPort>> movingEndpoints = new HashSet<>();
 
     /* Use this lock for manipulating the token map */
     private final ReadWriteLock lock = new ReentrantReadWriteLock(true);
@@ -100,26 +99,18 @@ public class TokenMetadata
     private final Topology topology;
     public final IPartitioner partitioner;
 
-    private static final Comparator<InetAddress> inetaddressCmp = new Comparator<InetAddress>()
-    {
-        public int compare(InetAddress o1, InetAddress o2)
-        {
-            return ByteBuffer.wrap(o1.getAddress()).compareTo(ByteBuffer.wrap(o2.getAddress()));
-        }
-    };
-
     // signals replication strategies that nodes have joined or left the ring and they need to recompute ownership
     private volatile long ringVersion = 0;
 
     public TokenMetadata()
     {
-        this(SortedBiMultiValMap.<Token, InetAddress>create(null, inetaddressCmp),
-             HashBiMap.<InetAddress, UUID>create(),
+        this(SortedBiMultiValMap.<Token, InetAddressAndPort>create(),
+             HashBiMap.create(),
              new Topology(),
              DatabaseDescriptor.getPartitioner());
     }
 
-    private TokenMetadata(BiMultiValMap<Token, InetAddress> tokenToEndpointMap, BiMap<InetAddress, UUID> endpointsMap, Topology topology, IPartitioner partitioner)
+    private TokenMetadata(BiMultiValMap<Token, InetAddressAndPort> tokenToEndpointMap, BiMap<InetAddressAndPort, UUID> endpointsMap, Topology topology, IPartitioner partitioner)
     {
         this.tokenToEndpointMap = tokenToEndpointMap;
         this.topology = topology;
@@ -143,7 +134,7 @@ public class TokenMetadata
     }
 
     /** @return the number of nodes bootstrapping into source's primary range */
-    public int pendingRangeChanges(InetAddress source)
+    public int pendingRangeChanges(InetAddressAndPort source)
     {
         int n = 0;
         Collection<Range<Token>> sourceRanges = getPrimaryRangesFor(getTokens(source));
@@ -165,14 +156,14 @@ public class TokenMetadata
     /**
      * Update token map with a single token/endpoint pair in normal state.
      */
-    public void updateNormalToken(Token token, InetAddress endpoint)
+    public void updateNormalToken(Token token, InetAddressAndPort endpoint)
     {
         updateNormalTokens(Collections.singleton(token), endpoint);
     }
 
-    public void updateNormalTokens(Collection<Token> tokens, InetAddress endpoint)
+    public void updateNormalTokens(Collection<Token> tokens, InetAddressAndPort endpoint)
     {
-        Multimap<InetAddress, Token> endpointTokens = HashMultimap.create();
+        Multimap<InetAddressAndPort, Token> endpointTokens = HashMultimap.create();
         for (Token token : tokens)
             endpointTokens.put(endpoint, token);
         updateNormalTokens(endpointTokens);
@@ -184,7 +175,7 @@ public class TokenMetadata
      * Prefer this whenever there are multiple pairs to update, as each update (whether a single or multiple)
      * is expensive (CASSANDRA-3831).
      */
-    public void updateNormalTokens(Multimap<InetAddress, Token> endpointTokens)
+    public void updateNormalTokens(Multimap<InetAddressAndPort, Token> endpointTokens)
     {
         if (endpointTokens.isEmpty())
             return;
@@ -193,7 +184,7 @@ public class TokenMetadata
         try
         {
             boolean shouldSortTokens = false;
-            for (InetAddress endpoint : endpointTokens.keySet())
+            for (InetAddressAndPort endpoint : endpointTokens.keySet())
             {
                 Collection<Token> tokens = endpointTokens.get(endpoint);
 
@@ -208,7 +199,7 @@ public class TokenMetadata
 
                 for (Token token : tokens)
                 {
-                    InetAddress prev = tokenToEndpointMap.put(token, endpoint);
+                    InetAddressAndPort prev = tokenToEndpointMap.put(token, endpoint);
                     if (!endpoint.equals(prev))
                     {
                         if (prev != null)
@@ -231,7 +222,7 @@ public class TokenMetadata
      * Store an end-point to host ID mapping.  Each ID must be unique, and
      * cannot be changed after the fact.
      */
-    public void updateHostId(UUID hostId, InetAddress endpoint)
+    public void updateHostId(UUID hostId, InetAddressAndPort endpoint)
     {
         assert hostId != null;
         assert endpoint != null;
@@ -239,7 +230,7 @@ public class TokenMetadata
         lock.writeLock().lock();
         try
         {
-            InetAddress storedEp = endpointToHostIdMap.inverse().get(hostId);
+            InetAddressAndPort storedEp = endpointToHostIdMap.inverse().get(hostId);
             if (storedEp != null)
             {
                 if (!storedEp.equals(endpoint) && (FailureDetector.instance.isAlive(storedEp)))
@@ -265,7 +256,7 @@ public class TokenMetadata
     }
 
     /** Return the unique host ID for an end-point. */
-    public UUID getHostId(InetAddress endpoint)
+    public UUID getHostId(InetAddressAndPort endpoint)
     {
         lock.readLock().lock();
         try
@@ -279,7 +270,7 @@ public class TokenMetadata
     }
 
     /** Return the end-point for a unique host ID */
-    public InetAddress getEndpointForHostId(UUID hostId)
+    public InetAddressAndPort getEndpointForHostId(UUID hostId)
     {
         lock.readLock().lock();
         try
@@ -293,12 +284,12 @@ public class TokenMetadata
     }
 
     /** @return a copy of the endpoint-to-id map for read-only operations */
-    public Map<InetAddress, UUID> getEndpointToHostIdMapForReading()
+    public Map<InetAddressAndPort, UUID> getEndpointToHostIdMapForReading()
     {
         lock.readLock().lock();
         try
         {
-            Map<InetAddress, UUID> readMap = new HashMap<>();
+            Map<InetAddressAndPort, UUID> readMap = new HashMap<>();
             readMap.putAll(endpointToHostIdMap);
             return readMap;
         }
@@ -309,17 +300,17 @@ public class TokenMetadata
     }
 
     @Deprecated
-    public void addBootstrapToken(Token token, InetAddress endpoint)
+    public void addBootstrapToken(Token token, InetAddressAndPort endpoint)
     {
         addBootstrapTokens(Collections.singleton(token), endpoint);
     }
 
-    public void addBootstrapTokens(Collection<Token> tokens, InetAddress endpoint)
+    public void addBootstrapTokens(Collection<Token> tokens, InetAddressAndPort endpoint)
     {
         addBootstrapTokens(tokens, endpoint, null);
     }
 
-    private void addBootstrapTokens(Collection<Token> tokens, InetAddress endpoint, InetAddress original)
+    private void addBootstrapTokens(Collection<Token> tokens, InetAddressAndPort endpoint, InetAddressAndPort original)
     {
         assert tokens != null && !tokens.isEmpty();
         assert endpoint != null;
@@ -328,7 +319,7 @@ public class TokenMetadata
         try
         {
 
-            InetAddress oldEndpoint;
+            InetAddressAndPort oldEndpoint;
 
             for (Token token : tokens)
             {
@@ -352,7 +343,7 @@ public class TokenMetadata
         }
     }
 
-    public void addReplaceTokens(Collection<Token> replacingTokens, InetAddress newNode, InetAddress oldNode)
+    public void addReplaceTokens(Collection<Token> replacingTokens, InetAddressAndPort newNode, InetAddressAndPort oldNode)
     {
         assert replacingTokens != null && !replacingTokens.isEmpty();
         assert newNode != null && oldNode != null;
@@ -379,12 +370,12 @@ public class TokenMetadata
         }
     }
 
-    public Optional<InetAddress> getReplacementNode(InetAddress endpoint)
+    public Optional<InetAddressAndPort> getReplacementNode(InetAddressAndPort endpoint)
     {
         return Optional.ofNullable(replacementToOriginal.inverse().get(endpoint));
     }
 
-    public Optional<InetAddress> getReplacingNode(InetAddress endpoint)
+    public Optional<InetAddressAndPort> getReplacingNode(InetAddressAndPort endpoint)
     {
         return Optional.ofNullable((replacementToOriginal.get(endpoint)));
     }
@@ -405,7 +396,7 @@ public class TokenMetadata
         }
     }
 
-    public void addLeavingEndpoint(InetAddress endpoint)
+    public void addLeavingEndpoint(InetAddressAndPort endpoint)
     {
         assert endpoint != null;
 
@@ -425,7 +416,7 @@ public class TokenMetadata
      * @param token token which is node moving to
      * @param endpoint address of the moving node
      */
-    public void addMovingEndpoint(Token token, InetAddress endpoint)
+    public void addMovingEndpoint(Token token, InetAddressAndPort endpoint)
     {
         assert endpoint != null;
 
@@ -441,7 +432,7 @@ public class TokenMetadata
         }
     }
 
-    public void removeEndpoint(InetAddress endpoint)
+    public void removeEndpoint(InetAddressAndPort endpoint)
     {
         assert endpoint != null;
 
@@ -469,7 +460,7 @@ public class TokenMetadata
     /**
      * This is called when the snitch properties for this endpoint are updated, see CASSANDRA-10238.
      */
-    public void updateTopology(InetAddress endpoint)
+    public void updateTopology(InetAddressAndPort endpoint)
     {
         assert endpoint != null;
 
@@ -509,14 +500,14 @@ public class TokenMetadata
      * Remove pair of token/address from moving endpoints
      * @param endpoint address of the moving node
      */
-    public void removeFromMoving(InetAddress endpoint)
+    public void removeFromMoving(InetAddressAndPort endpoint)
     {
         assert endpoint != null;
 
         lock.writeLock().lock();
         try
         {
-            for (Pair<Token, InetAddress> pair : movingEndpoints)
+            for (Pair<Token, InetAddressAndPort> pair : movingEndpoints)
             {
                 if (pair.right.equals(endpoint))
                 {
@@ -533,7 +524,7 @@ public class TokenMetadata
         }
     }
 
-    public Collection<Token> getTokens(InetAddress endpoint)
+    public Collection<Token> getTokens(InetAddressAndPort endpoint)
     {
         assert endpoint != null;
         assert isMember(endpoint); // don't want to return nulls
@@ -550,12 +541,12 @@ public class TokenMetadata
     }
 
     @Deprecated
-    public Token getToken(InetAddress endpoint)
+    public Token getToken(InetAddressAndPort endpoint)
     {
         return getTokens(endpoint).iterator().next();
     }
 
-    public boolean isMember(InetAddress endpoint)
+    public boolean isMember(InetAddressAndPort endpoint)
     {
         assert endpoint != null;
 
@@ -570,7 +561,7 @@ public class TokenMetadata
         }
     }
 
-    public boolean isLeaving(InetAddress endpoint)
+    public boolean isLeaving(InetAddressAndPort endpoint)
     {
         assert endpoint != null;
 
@@ -585,7 +576,7 @@ public class TokenMetadata
         }
     }
 
-    public boolean isMoving(InetAddress endpoint)
+    public boolean isMoving(InetAddressAndPort endpoint)
     {
         assert endpoint != null;
 
@@ -593,7 +584,7 @@ public class TokenMetadata
 
         try
         {
-            for (Pair<Token, InetAddress> pair : movingEndpoints)
+            for (Pair<Token, InetAddressAndPort> pair : movingEndpoints)
             {
                 if (pair.right.equals(endpoint))
                     return true;
@@ -618,7 +609,7 @@ public class TokenMetadata
         lock.readLock().lock();
         try
         {
-            return new TokenMetadata(SortedBiMultiValMap.create(tokenToEndpointMap, null, inetaddressCmp),
+            return new TokenMetadata(SortedBiMultiValMap.create(tokenToEndpointMap),
                                      HashBiMap.create(endpointToHostIdMap),
                                      new Topology(topology),
                                      partitioner);
@@ -673,9 +664,9 @@ public class TokenMetadata
         }
     }
 
-    private static TokenMetadata removeEndpoints(TokenMetadata allLeftMetadata, Set<InetAddress> leavingEndpoints)
+    private static TokenMetadata removeEndpoints(TokenMetadata allLeftMetadata, Set<InetAddressAndPort> leavingEndpoints)
     {
-        for (InetAddress endpoint : leavingEndpoints)
+        for (InetAddressAndPort endpoint : leavingEndpoints)
             allLeftMetadata.removeEndpoint(endpoint);
 
         return allLeftMetadata;
@@ -695,11 +686,11 @@ public class TokenMetadata
         {
             TokenMetadata metadata = cloneOnlyTokenMap();
 
-            for (InetAddress endpoint : leavingEndpoints)
+            for (InetAddressAndPort endpoint : leavingEndpoints)
                 metadata.removeEndpoint(endpoint);
 
 
-            for (Pair<Token, InetAddress> pair : movingEndpoints)
+            for (Pair<Token, InetAddressAndPort> pair : movingEndpoints)
                 metadata.updateNormalToken(pair.left, pair.right);
 
             return metadata;
@@ -710,7 +701,7 @@ public class TokenMetadata
         }
     }
 
-    public InetAddress getEndpoint(Token token)
+    public InetAddressAndPort getEndpoint(Token token)
     {
         lock.readLock().lock();
         try
@@ -742,17 +733,17 @@ public class TokenMetadata
         return sortedTokens;
     }
 
-    public Multimap<Range<Token>, InetAddress> getPendingRangesMM(String keyspaceName)
+    public Multimap<Range<Token>, InetAddressAndPort> getPendingRangesMM(String keyspaceName)
     {
-        Multimap<Range<Token>, InetAddress> map = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> map = HashMultimap.create();
         PendingRangeMaps pendingRangeMaps = this.pendingRanges.get(keyspaceName);
 
         if (pendingRangeMaps != null)
         {
-            for (Map.Entry<Range<Token>, List<InetAddress>> entry : pendingRangeMaps)
+            for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : pendingRangeMaps)
             {
                 Range<Token> range = entry.getKey();
-                for (InetAddress address : entry.getValue())
+                for (InetAddressAndPort address : entry.getValue())
                 {
                     map.put(range, address);
                 }
@@ -768,10 +759,10 @@ public class TokenMetadata
         return this.pendingRanges.get(keyspaceName);
     }
 
-    public List<Range<Token>> getPendingRanges(String keyspaceName, InetAddress endpoint)
+    public List<Range<Token>> getPendingRanges(String keyspaceName, InetAddressAndPort endpoint)
     {
         List<Range<Token>> ranges = new ArrayList<>();
-        for (Map.Entry<Range<Token>, InetAddress> entry : getPendingRangesMM(keyspaceName).entries())
+        for (Map.Entry<Range<Token>, InetAddressAndPort> entry : getPendingRangesMM(keyspaceName).entries())
         {
             if (entry.getValue().equals(endpoint))
             {
@@ -824,9 +815,9 @@ public class TokenMetadata
                 long startedAt = System.currentTimeMillis();
 
                 // create clone of current state
-                BiMultiValMap<Token, InetAddress> bootstrapTokensClone = new BiMultiValMap<>();
-                Set<InetAddress> leavingEndpointsClone = new HashSet<>();
-                Set<Pair<Token, InetAddress>> movingEndpointsClone = new HashSet<>();
+                BiMultiValMap<Token, InetAddressAndPort> bootstrapTokensClone = new BiMultiValMap<>();
+                Set<InetAddressAndPort> leavingEndpointsClone = new HashSet<>();
+                Set<Pair<Token, InetAddressAndPort>> movingEndpointsClone = new HashSet<>();
                 TokenMetadata metadata;
 
                 lock.readLock().lock();
@@ -859,29 +850,29 @@ public class TokenMetadata
      */
     private static PendingRangeMaps calculatePendingRanges(AbstractReplicationStrategy strategy,
                                                            TokenMetadata metadata,
-                                                           BiMultiValMap<Token, InetAddress> bootstrapTokens,
-                                                           Set<InetAddress> leavingEndpoints,
-                                                           Set<Pair<Token, InetAddress>> movingEndpoints)
+                                                           BiMultiValMap<Token, InetAddressAndPort> bootstrapTokens,
+                                                           Set<InetAddressAndPort> leavingEndpoints,
+                                                           Set<Pair<Token, InetAddressAndPort>> movingEndpoints)
     {
         PendingRangeMaps newPendingRanges = new PendingRangeMaps();
 
-        Multimap<InetAddress, Range<Token>> addressRanges = strategy.getAddressRanges(metadata);
+        Multimap<InetAddressAndPort, Range<Token>> addressRanges = strategy.getAddressRanges(metadata);
 
         // Copy of metadata reflecting the situation after all leave operations are finished.
         TokenMetadata allLeftMetadata = removeEndpoints(metadata.cloneOnlyTokenMap(), leavingEndpoints);
 
         // get all ranges that will be affected by leaving nodes
         Set<Range<Token>> affectedRanges = new HashSet<Range<Token>>();
-        for (InetAddress endpoint : leavingEndpoints)
+        for (InetAddressAndPort endpoint : leavingEndpoints)
             affectedRanges.addAll(addressRanges.get(endpoint));
 
         // for each of those ranges, find what new nodes will be responsible for the range when
         // all leaving nodes are gone.
         for (Range<Token> range : affectedRanges)
         {
-            Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
-            Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
-            for (InetAddress address : Sets.difference(newEndpoints, currentEndpoints))
+            Set<InetAddressAndPort> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
+            Set<InetAddressAndPort> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
+            for (InetAddressAndPort address : Sets.difference(newEndpoints, currentEndpoints))
             {
                 newPendingRanges.addPendingRange(range, address);
             }
@@ -892,8 +883,8 @@ public class TokenMetadata
 
         // For each of the bootstrapping nodes, simply add and remove them one by one to
         // allLeftMetadata and check in between what their ranges would be.
-        Multimap<InetAddress, Token> bootstrapAddresses = bootstrapTokens.inverse();
-        for (InetAddress endpoint : bootstrapAddresses.keySet())
+        Multimap<InetAddressAndPort, Token> bootstrapAddresses = bootstrapTokens.inverse();
+        for (InetAddressAndPort endpoint : bootstrapAddresses.keySet())
         {
             Collection<Token> tokens = bootstrapAddresses.get(endpoint);
 
@@ -910,11 +901,11 @@ public class TokenMetadata
 
         // For each of the moving nodes, we do the same thing we did for bootstrapping:
         // simply add and remove them one by one to allLeftMetadata and check in between what their ranges would be.
-        for (Pair<Token, InetAddress> moving : movingEndpoints)
+        for (Pair<Token, InetAddressAndPort> moving : movingEndpoints)
         {
             //Calculate all the ranges which will could be affected. This will include the ranges before and after the move.
             Set<Range<Token>> moveAffectedRanges = new HashSet<>();
-            InetAddress endpoint = moving.right; // address of the moving node
+            InetAddressAndPort endpoint = moving.right; // address of the moving node
             //Add ranges before the move
             for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
             {
@@ -930,10 +921,10 @@ public class TokenMetadata
 
             for(Range<Token> range : moveAffectedRanges)
             {
-                Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
-                Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
-                Set<InetAddress> difference = Sets.difference(newEndpoints, currentEndpoints);
-                for(final InetAddress address : difference)
+                Set<InetAddressAndPort> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
+                Set<InetAddressAndPort> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
+                Set<InetAddressAndPort> difference = Sets.difference(newEndpoints, currentEndpoints);
+                for(final InetAddressAndPort address : difference)
                 {
                     Collection<Range<Token>> newRanges = strategy.getAddressRanges(allLeftMetadata).get(address);
                     Collection<Range<Token>> oldRanges = strategy.getAddressRanges(metadata).get(address);
@@ -973,12 +964,12 @@ public class TokenMetadata
     }
 
     /** @return a copy of the bootstrapping tokens map */
-    public BiMultiValMap<Token, InetAddress> getBootstrapTokens()
+    public BiMultiValMap<Token, InetAddressAndPort> getBootstrapTokens()
     {
         lock.readLock().lock();
         try
         {
-            return new BiMultiValMap<Token, InetAddress>(bootstrapTokens);
+            return new BiMultiValMap<>(bootstrapTokens);
         }
         finally
         {
@@ -986,7 +977,7 @@ public class TokenMetadata
         }
     }
 
-    public Set<InetAddress> getAllEndpoints()
+    public Set<InetAddressAndPort> getAllEndpoints()
     {
         lock.readLock().lock();
         try
@@ -1010,7 +1001,7 @@ public class TokenMetadata
     }
 
     /** caller should not modify leavingEndpoints */
-    public Set<InetAddress> getLeavingEndpoints()
+    public Set<InetAddressAndPort> getLeavingEndpoints()
     {
         lock.readLock().lock();
         try
@@ -1037,7 +1028,7 @@ public class TokenMetadata
      * Endpoints which are migrating to the new tokens
      * @return set of addresses of moving endpoints
      */
-    public Set<Pair<Token, InetAddress>> getMovingEndpoints()
+    public Set<Pair<Token, InetAddressAndPort>> getMovingEndpoints()
     {
         lock.readLock().lock();
         try
@@ -1148,14 +1139,14 @@ public class TokenMetadata
         lock.readLock().lock();
         try
         {
-            Multimap<InetAddress, Token> endpointToTokenMap = tokenToEndpointMap.inverse();
-            Set<InetAddress> eps = endpointToTokenMap.keySet();
+            Multimap<InetAddressAndPort, Token> endpointToTokenMap = tokenToEndpointMap.inverse();
+            Set<InetAddressAndPort> eps = endpointToTokenMap.keySet();
 
             if (!eps.isEmpty())
             {
                 sb.append("Normal Tokens:");
                 sb.append(System.getProperty("line.separator"));
-                for (InetAddress ep : eps)
+                for (InetAddressAndPort ep : eps)
                 {
                     sb.append(ep);
                     sb.append(':');
@@ -1168,7 +1159,7 @@ public class TokenMetadata
             {
                 sb.append("Bootstrapping Tokens:" );
                 sb.append(System.getProperty("line.separator"));
-                for (Map.Entry<Token, InetAddress> entry : bootstrapTokens.entrySet())
+                for (Map.Entry<Token, InetAddressAndPort> entry : bootstrapTokens.entrySet())
                 {
                     sb.append(entry.getValue()).append(':').append(entry.getKey());
                     sb.append(System.getProperty("line.separator"));
@@ -1179,7 +1170,7 @@ public class TokenMetadata
             {
                 sb.append("Leaving Endpoints:");
                 sb.append(System.getProperty("line.separator"));
-                for (InetAddress ep : leavingEndpoints)
+                for (InetAddressAndPort ep : leavingEndpoints)
                 {
                     sb.append(ep);
                     sb.append(System.getProperty("line.separator"));
@@ -1213,7 +1204,7 @@ public class TokenMetadata
         return sb.toString();
     }
 
-    public Collection<InetAddress> pendingEndpointsFor(Token token, String keyspaceName)
+    public Collection<InetAddressAndPort> pendingEndpointsFor(Token token, String keyspaceName)
     {
         PendingRangeMaps pendingRangeMaps = this.pendingRanges.get(keyspaceName);
         if (pendingRangeMaps == null)
@@ -1225,19 +1216,19 @@ public class TokenMetadata
     /**
      * @deprecated retained for benefit of old tests
      */
-    public Collection<InetAddress> getWriteEndpoints(Token token, String keyspaceName, Collection<InetAddress> naturalEndpoints)
+    public Collection<InetAddressAndPort> getWriteEndpoints(Token token, String keyspaceName, Collection<InetAddressAndPort> naturalEndpoints)
     {
         return ImmutableList.copyOf(Iterables.concat(naturalEndpoints, pendingEndpointsFor(token, keyspaceName)));
     }
 
     /** @return an endpoint to token multimap representation of tokenToEndpointMap (a copy) */
-    public Multimap<InetAddress, Token> getEndpointToTokenMapForReading()
+    public Multimap<InetAddressAndPort, Token> getEndpointToTokenMapForReading()
     {
         lock.readLock().lock();
         try
         {
-            Multimap<InetAddress, Token> cloned = HashMultimap.create();
-            for (Map.Entry<Token, InetAddress> entry : tokenToEndpointMap.entrySet())
+            Multimap<InetAddressAndPort, Token> cloned = HashMultimap.create();
+            for (Map.Entry<Token, InetAddressAndPort> entry : tokenToEndpointMap.entrySet())
                 cloned.put(entry.getValue(), entry.getKey());
             return cloned;
         }
@@ -1251,12 +1242,12 @@ public class TokenMetadata
      * @return a (stable copy, won't be modified) Token to Endpoint map for all the normal and bootstrapping nodes
      *         in the cluster.
      */
-    public Map<Token, InetAddress> getNormalAndBootstrappingTokenToEndpointMap()
+    public Map<Token, InetAddressAndPort> getNormalAndBootstrappingTokenToEndpointMap()
     {
         lock.readLock().lock();
         try
         {
-            Map<Token, InetAddress> map = new HashMap<>(tokenToEndpointMap.size() + bootstrapTokens.size());
+            Map<Token, InetAddressAndPort> map = new HashMap<>(tokenToEndpointMap.size() + bootstrapTokens.size());
             map.putAll(tokenToEndpointMap);
             map.putAll(bootstrapTokens);
             return map;
@@ -1302,11 +1293,11 @@ public class TokenMetadata
     public static class Topology
     {
         /** multi-map of DC to endpoints in that DC */
-        private final Multimap<String, InetAddress> dcEndpoints;
+        private final Multimap<String, InetAddressAndPort> dcEndpoints;
         /** map of DC to multi-map of rack to endpoints in that rack */
-        private final Map<String, Multimap<String, InetAddress>> dcRacks;
+        private final Map<String, Multimap<String, InetAddressAndPort>> dcRacks;
         /** reverse-lookup map for endpoint to current known dc/rack assignment */
-        private final Map<InetAddress, Pair<String, String>> currentLocations;
+        private final Map<InetAddressAndPort, Pair<String, String>> currentLocations;
 
         Topology()
         {
@@ -1337,7 +1328,7 @@ public class TokenMetadata
         /**
          * Stores current DC/rack assignment for ep
          */
-        void addEndpoint(InetAddress ep)
+        void addEndpoint(InetAddressAndPort ep)
         {
             IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
             String dc = snitch.getDatacenter(ep);
@@ -1353,12 +1344,12 @@ public class TokenMetadata
             doAddEndpoint(ep, dc, rack);
         }
 
-        private void doAddEndpoint(InetAddress ep, String dc, String rack)
+        private void doAddEndpoint(InetAddressAndPort ep, String dc, String rack)
         {
             dcEndpoints.put(dc, ep);
 
             if (!dcRacks.containsKey(dc))
-                dcRacks.put(dc, HashMultimap.<String, InetAddress>create());
+                dcRacks.put(dc, HashMultimap.create());
             dcRacks.get(dc).put(rack, ep);
 
             currentLocations.put(ep, Pair.create(dc, rack));
@@ -1367,7 +1358,7 @@ public class TokenMetadata
         /**
          * Removes current DC/rack assignment for ep
          */
-        void removeEndpoint(InetAddress ep)
+        void removeEndpoint(InetAddressAndPort ep)
         {
             if (!currentLocations.containsKey(ep))
                 return;
@@ -1375,13 +1366,13 @@ public class TokenMetadata
             doRemoveEndpoint(ep, currentLocations.remove(ep));
         }
 
-        private void doRemoveEndpoint(InetAddress ep, Pair<String, String> current)
+        private void doRemoveEndpoint(InetAddressAndPort ep, Pair<String, String> current)
         {
             dcRacks.get(current.left).remove(current.right, ep);
             dcEndpoints.remove(current.left, ep);
         }
 
-        void updateEndpoint(InetAddress ep)
+        void updateEndpoint(InetAddressAndPort ep)
         {
             IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
             if (snitch == null || !currentLocations.containsKey(ep))
@@ -1396,11 +1387,11 @@ public class TokenMetadata
             if (snitch == null)
                 return;
 
-            for (InetAddress ep : currentLocations.keySet())
+            for (InetAddressAndPort ep : currentLocations.keySet())
                 updateEndpoint(ep, snitch);
         }
 
-        private void updateEndpoint(InetAddress ep, IEndpointSnitch snitch)
+        private void updateEndpoint(InetAddressAndPort ep, IEndpointSnitch snitch)
         {
             Pair<String, String> current = currentLocations.get(ep);
             String dc = snitch.getDatacenter(ep);
@@ -1415,7 +1406,7 @@ public class TokenMetadata
         /**
          * @return multi-map of DC to endpoints in that DC
          */
-        public Multimap<String, InetAddress> getDatacenterEndpoints()
+        public Multimap<String, InetAddressAndPort> getDatacenterEndpoints()
         {
             return dcEndpoints;
         }
@@ -1423,7 +1414,7 @@ public class TokenMetadata
         /**
          * @return map of DC to multi-map of rack to endpoints in that rack
          */
-        public Map<String, Multimap<String, InetAddress>> getDatacenterRacks()
+        public Map<String, Multimap<String, InetAddressAndPort>> getDatacenterRacks()
         {
             return dcRacks;
         }
@@ -1431,7 +1422,7 @@ public class TokenMetadata
         /**
          * @return The DC and rack of the given endpoint.
          */
-        public Pair<String, String> getLocation(InetAddress addr)
+        public Pair<String, String> getLocation(InetAddressAndPort addr)
         {
             return currentLocations.get(addr);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 7815784..3655a40 100644
--- a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
@@ -17,14 +17,14 @@
  */
 package org.apache.cassandra.metrics;
 
-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.locator.InetAddressAndPort;
+
 /**
  * Metrics for internode connections.
  */
@@ -65,10 +65,10 @@ public class ConnectionMetrics
      *
      * @param ip IP address to use for metrics label
      */
-    public ConnectionMetrics(InetAddress ip, final OutboundMessagingPool messagingPool)
+    public ConnectionMetrics(InetAddressAndPort ip, final OutboundMessagingPool messagingPool)
     {
         // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName
-        address = ip.getHostAddress().replace(':', '.');
+        address = ip.toString().replace(':', '.');
 
         factory = new DefaultNameFactory("Connection", address);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
index 052830a..56888da 100644
--- a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.net.InetAddress;
 import java.util.Map.Entry;
 
 import com.google.common.util.concurrent.MoreExecutors;
@@ -27,6 +26,7 @@ import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
 
 import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.UUIDGen;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,28 +43,28 @@ public class HintedHandoffMetrics
     private static final MetricNameFactory factory = new DefaultNameFactory("HintedHandOffManager");
 
     /** Total number of hints which are not stored, This is not a cache. */
-    private final LoadingCache<InetAddress, DifferencingCounter> notStored = Caffeine.newBuilder()
-         .executor(MoreExecutors.directExecutor())
-         .build(DifferencingCounter::new);
+    private final LoadingCache<InetAddressAndPort, DifferencingCounter> notStored = Caffeine.newBuilder()
+                                                                                            .executor(MoreExecutors.directExecutor())
+                                                                                            .build(DifferencingCounter::new);
 
     /** Total number of hints that have been created, This is not a cache. */
-    private final LoadingCache<InetAddress, Counter> createdHintCounts = Caffeine.newBuilder()
-         .executor(MoreExecutors.directExecutor())
-         .build(address -> Metrics.counter(factory.createMetricName("Hints_created-" + address.getHostAddress().replace(':', '.'))));
+    private final LoadingCache<InetAddressAndPort, Counter> createdHintCounts = Caffeine.newBuilder()
+                                                                                        .executor(MoreExecutors.directExecutor())
+                                                                                        .build(address -> Metrics.counter(factory.createMetricName("Hints_created-" + address.toString().replace(':', '.'))));
 
-    public void incrCreatedHints(InetAddress address)
+    public void incrCreatedHints(InetAddressAndPort address)
     {
         createdHintCounts.get(address).inc();
     }
 
-    public void incrPastWindow(InetAddress address)
+    public void incrPastWindow(InetAddressAndPort address)
     {
         notStored.get(address).mark();
     }
 
     public void log()
     {
-        for (Entry<InetAddress, DifferencingCounter> entry : notStored.asMap().entrySet())
+        for (Entry<InetAddressAndPort, DifferencingCounter> entry : notStored.asMap().entrySet())
         {
             long difference = entry.getValue().difference();
             if (difference == 0)
@@ -79,9 +79,10 @@ public class HintedHandoffMetrics
         private final Counter meter;
         private long reported = 0;
 
-        public DifferencingCounter(InetAddress address)
+        public DifferencingCounter(InetAddressAndPort address)
         {
-            this.meter = Metrics.counter(factory.createMetricName("Hints_not_stored-" + address.getHostAddress().replace(':', '.')));
+            //This changes the name of the metric, people can update their monitoring when upgrading?
+            this.meter = Metrics.counter(factory.createMetricName("Hints_not_stored-" + address.toString().replace(':', '.')));
         }
 
         public long difference()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java b/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java
index d6a75f7..424f502 100644
--- a/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.net.InetAddress;
-
 import com.google.common.util.concurrent.MoreExecutors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -27,6 +25,7 @@ import com.codahale.metrics.Histogram;
 import com.codahale.metrics.Meter;
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
@@ -47,11 +46,11 @@ public final class HintsServiceMetrics
     private static final Histogram globalDelayHistogram = Metrics.histogram(factory.createMetricName("Hint_delays"), false);
 
     /** Histograms per-endpoint of hint delivery delays, This is not a cache. */
-    private static final LoadingCache<InetAddress, Histogram> delayByEndpoint = Caffeine.newBuilder()
-        .executor(MoreExecutors.directExecutor())
-        .build(address -> Metrics.histogram(factory.createMetricName("Hint_delays-"+address.getHostAddress().replace(':', '.')), false));
+    private static final LoadingCache<InetAddressAndPort, Histogram> delayByEndpoint = Caffeine.newBuilder()
+                                                                                               .executor(MoreExecutors.directExecutor())
+                                                                                               .build(address -> Metrics.histogram(factory.createMetricName("Hint_delays-"+address.toString().replace(':', '.')), false));
 
-    public static void updateDelayMetrics(InetAddress endpoint, long delay)
+    public static void updateDelayMetrics(InetAddressAndPort endpoint, long delay)
     {
         if (delay <= 0)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/metrics/MessagingMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java
index 5f640b9..2f096f6 100644
--- a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.net.InetAddress;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
@@ -26,6 +25,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.codahale.metrics.Timer;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
@@ -47,7 +47,7 @@ public class MessagingMetrics
         queueWaitLatency = new ConcurrentHashMap<>();
     }
 
-    public void addTimeTaken(InetAddress from, long timeTaken)
+    public void addTimeTaken(InetAddressAndPort from, long timeTaken)
     {
         String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(from);
         Timer timer = dcLatency.get(dc);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/StreamingMetrics.java b/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
index 72e9b23..d220ca5 100644
--- a/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
@@ -17,11 +17,12 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.net.InetAddress;
 import java.util.concurrent.ConcurrentMap;
 
 
 import com.codahale.metrics.Counter;
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
 import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
@@ -33,7 +34,7 @@ public class StreamingMetrics
 {
     public static final String TYPE_NAME = "Streaming";
 
-    private static final ConcurrentMap<InetAddress, StreamingMetrics> instances = new NonBlockingHashMap<InetAddress, StreamingMetrics>();
+    private static final ConcurrentMap<InetAddressAndPort, StreamingMetrics> instances = new NonBlockingHashMap<>();
 
     public static final Counter activeStreamsOutbound = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "ActiveOutboundStreams", null));
     public static final Counter totalIncomingBytes = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalIncomingBytes", null));
@@ -41,7 +42,7 @@ public class StreamingMetrics
     public final Counter incomingBytes;
     public final Counter outgoingBytes;
 
-    public static StreamingMetrics get(InetAddress ip)
+    public static StreamingMetrics get(InetAddressAndPort ip)
     {
        StreamingMetrics metrics = instances.get(ip);
        if (metrics == null)
@@ -52,9 +53,9 @@ public class StreamingMetrics
        return metrics;
     }
 
-    public StreamingMetrics(final InetAddress peer)
+    public StreamingMetrics(final InetAddressAndPort peer)
     {
-        MetricNameFactory factory = new DefaultNameFactory("Streaming", peer.getHostAddress().replace(':', '.'));
+        MetricNameFactory factory = new DefaultNameFactory("Streaming", peer.toString().replace(':', '.'));
         incomingBytes = Metrics.counter(factory.createMetricName("IncomingBytes"));
         outgoingBytes= Metrics.counter(factory.createMetricName("OutgoingBytes"));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/BackPressureState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/BackPressureState.java b/src/java/org/apache/cassandra/net/BackPressureState.java
index 34fd0dd..886c075 100644
--- a/src/java/org/apache/cassandra/net/BackPressureState.java
+++ b/src/java/org/apache/cassandra/net/BackPressureState.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Interface meant to track the back-pressure state per replica host.
@@ -47,5 +47,5 @@ public interface BackPressureState
     /**
      * Returns the host this state refers to.
      */
-    InetAddress getHost();
+    InetAddressAndPort getHost();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/BackPressureStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/BackPressureStrategy.java b/src/java/org/apache/cassandra/net/BackPressureStrategy.java
index 78f748b..6b49495 100644
--- a/src/java/org/apache/cassandra/net/BackPressureStrategy.java
+++ b/src/java/org/apache/cassandra/net/BackPressureStrategy.java
@@ -17,10 +17,11 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 /**
  * Back-pressure algorithm interface.
  * <p>
@@ -39,5 +40,5 @@ public interface BackPressureStrategy<S extends BackPressureState>
     /**
      * Creates a new {@link BackPressureState} initialized as needed by the specific implementation.
      */
-    S newState(InetAddress host);
+    S newState(InetAddressAndPort host);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/CallbackInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/CallbackInfo.java b/src/java/org/apache/cassandra/net/CallbackInfo.java
index ea000ae..f2ed8a1 100644
--- a/src/java/org/apache/cassandra/net/CallbackInfo.java
+++ b/src/java/org/apache/cassandra/net/CallbackInfo.java
@@ -17,9 +17,8 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
-
 import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Encapsulates the callback information.
@@ -28,7 +27,7 @@ import org.apache.cassandra.io.IVersionedSerializer;
  */
 public class CallbackInfo
 {
-    protected final InetAddress target;
+    protected final InetAddressAndPort target;
     protected final IAsyncCallback callback;
     protected final IVersionedSerializer<?> serializer;
     private final boolean failureCallback;
@@ -41,7 +40,7 @@ public class CallbackInfo
      * @param serializer serializer to deserialize response message
      * @param failureCallback True when we have a callback to handle failures
      */
-    public CallbackInfo(InetAddress target, IAsyncCallback callback, IVersionedSerializer<?> serializer, boolean failureCallback)
+    public CallbackInfo(InetAddressAndPort target, IAsyncCallback callback, IVersionedSerializer<?> serializer, boolean failureCallback)
     {
         this.target = target;
         this.callback = callback;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java b/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
index 83bbbf3..b58ca47 100644
--- a/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
+++ b/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
@@ -21,28 +21,108 @@ import java.io.*;
 import java.net.Inet4Address;
 import java.net.Inet6Address;
 import java.net.InetAddress;
+import java.nio.ByteBuffer;
 
-public class CompactEndpointSerializationHelper
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+
+/*
+ * As of version 4.0 the endpoint description includes a port number as an unsigned short
+ */
+public class CompactEndpointSerializationHelper implements IVersionedSerializer<InetAddressAndPort>
 {
-    public static void serialize(InetAddress endpoint, DataOutput out) throws IOException
+    public static final IVersionedSerializer<InetAddressAndPort> instance = new CompactEndpointSerializationHelper();
+
+    /**
+     * Streaming uses its own version numbering so we need to ignore it and always use currrent version.
+     * There is no cross version streaming so it will always use the latest address serialization.
+     **/
+    public static final IVersionedSerializer<InetAddressAndPort> streamingInstance = new IVersionedSerializer<InetAddressAndPort>()
     {
-        byte[] buf = endpoint.getAddress();
-        out.writeByte(buf.length);
-        out.write(buf);
+        public void serialize(InetAddressAndPort inetAddressAndPort, DataOutputPlus out, int version) throws IOException
+        {
+            instance.serialize(inetAddressAndPort, out, MessagingService.current_version);
+        }
+
+        public InetAddressAndPort deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return instance.deserialize(in, MessagingService.current_version);
+        }
+
+        public long serializedSize(InetAddressAndPort inetAddressAndPort, int version)
+        {
+            return instance.serializedSize(inetAddressAndPort, MessagingService.current_version);
+        }
+    };
+
+    private CompactEndpointSerializationHelper() {}
+
+    public void serialize(InetAddressAndPort endpoint, DataOutputPlus out, int version) throws IOException
+    {
+        if (version >= MessagingService.VERSION_40)
+        {
+            byte[] buf = endpoint.addressBytes;
+            out.writeByte(buf.length + 2);
+            out.write(buf);
+            out.writeShort(endpoint.port);
+        }
+        else
+        {
+            byte[] buf = endpoint.addressBytes;
+            out.writeByte(buf.length);
+            out.write(buf);
+        }
     }
 
-    public static InetAddress deserialize(DataInput in) throws IOException
+    public InetAddressAndPort deserialize(DataInputPlus in, int version) throws IOException
     {
-        byte[] bytes = new byte[in.readByte()];
-        in.readFully(bytes, 0, bytes.length);
-        return InetAddress.getByAddress(bytes);
+        int size = in.readByte() & 0xFF;
+        switch(size)
+        {
+            //The original pre-4.0 serialiation of just an address
+            case 4:
+            case 16:
+            {
+                byte[] bytes = new byte[size];
+                in.readFully(bytes, 0, bytes.length);
+                return InetAddressAndPort.getByAddress(bytes);
+            }
+            //Address and one port
+            case 6:
+            case 18:
+            {
+                byte[] bytes = new byte[size - 2];
+                in.readFully(bytes);
+
+                int port = in.readShort() & 0xFFFF;
+                return InetAddressAndPort.getByAddressOverrideDefaults(InetAddress.getByAddress(bytes), bytes, port);
+            }
+            default:
+                throw new AssertionError("Unexpected size " + size);
+
+        }
     }
 
-    public static int serializedSize(InetAddress from)
+    public long serializedSize(InetAddressAndPort from, int version)
     {
-        if (from instanceof Inet4Address)
-            return 1 + 4;
-        assert from instanceof Inet6Address;
-        return 1 + 16;
+        //4.0 includes a port number
+        if (version >= MessagingService.VERSION_40)
+        {
+            if (from.address instanceof Inet4Address)
+                return 1 + 4 + 2;
+            assert from.address instanceof Inet6Address;
+            return 1 + 16 + 2;
+        }
+        else
+        {
+            if (from.address instanceof Inet4Address)
+                return 1 + 4;
+            assert from.address instanceof Inet6Address;
+            return 1 + 16;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/ForwardToContainer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/ForwardToContainer.java b/src/java/org/apache/cassandra/net/ForwardToContainer.java
new file mode 100644
index 0000000..ac9e725
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/ForwardToContainer.java
@@ -0,0 +1,43 @@
+/*
+ * 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.util.Collection;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+/**
+ * Contains forward to information until it can be serialized as part of a message using a version
+ * specific serialization
+ */
+public class ForwardToContainer
+{
+    public final Collection<InetAddressAndPort> targets;
+    public final int[] messageIds;
+
+    public ForwardToContainer(Collection<InetAddressAndPort> targets,
+                              int[] messageIds)
+    {
+        Preconditions.checkArgument(targets.size() == messageIds.length);
+        this.targets = targets;
+        this.messageIds = messageIds;
+    }
+}


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


[07/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/LoaderOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/LoaderOptions.java b/src/java/org/apache/cassandra/tools/LoaderOptions.java
index c821e6a..4646ba4 100644
--- a/src/java/org/apache/cassandra/tools/LoaderOptions.java
+++ b/src/java/org/apache/cassandra/tools/LoaderOptions.java
@@ -27,8 +27,12 @@ import java.net.*;
 import java.util.HashSet;
 import java.util.Set;
 
+import com.google.common.base.Throwables;
+import com.google.common.net.HostAndPort;
+
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 
 import com.datastax.driver.core.AuthProvider;
@@ -54,6 +58,7 @@ public class LoaderOptions
     public static final String THROTTLE_MBITS = "throttle";
     public static final String INTER_DC_THROTTLE_MBITS = "inter-dc-throttle";
     public static final String TOOL_NAME = "sstableloader";
+    public static final String ALLOW_SERVER_PORT_DISCOVERY_OPTION = "server-port-discovery";
 
     /* client encryption options */
     public static final String SSL_TRUSTSTORE = "truststore";
@@ -80,8 +85,9 @@ public class LoaderOptions
     public final EncryptionOptions clientEncOptions;
     public final int connectionsPerHost;
     public final EncryptionOptions.ServerEncryptionOptions serverEncOptions;
-    public final Set<InetAddress> hosts;
-    public final Set<InetAddress> ignores;
+    public final Set<InetSocketAddress> hosts;
+    public final Set<InetAddressAndPort> ignores;
+    public final boolean allowServerPortDiscovery;
 
     LoaderOptions(Builder builder)
     {
@@ -101,6 +107,7 @@ public class LoaderOptions
         connectionsPerHost = builder.connectionsPerHost;
         serverEncOptions = builder.serverEncOptions;
         hosts = builder.hosts;
+        allowServerPortDiscovery = builder.allowServerPortDiscovery;
         ignores = builder.ignores;
     }
 
@@ -122,8 +129,11 @@ public class LoaderOptions
         EncryptionOptions clientEncOptions = new EncryptionOptions();
         int connectionsPerHost = 1;
         EncryptionOptions.ServerEncryptionOptions serverEncOptions = new EncryptionOptions.ServerEncryptionOptions();
-        Set<InetAddress> hosts = new HashSet<>();
-        Set<InetAddress> ignores = new HashSet<>();
+        Set<InetAddress> hostsArg = new HashSet<>();
+        Set<InetAddress> ignoresArg = new HashSet<>();
+        Set<InetSocketAddress> hosts = new HashSet<>();
+        Set<InetAddressAndPort> ignores = new HashSet<>();
+        boolean allowServerPortDiscovery;
 
         Builder()
         {
@@ -133,6 +143,23 @@ public class LoaderOptions
         public LoaderOptions build()
         {
             constructAuthProvider();
+
+            try
+            {
+                for (InetAddress host : hostsArg)
+                {
+                    hosts.add(new InetSocketAddress(host, nativePort));
+                }
+                for (InetAddress host : ignoresArg)
+                {
+                    ignores.add(InetAddressAndPort.getByNameOverrideDefaults(host.getHostAddress(), storagePort));
+                }
+            }
+            catch (UnknownHostException e)
+            {
+                Throwables.propagate(e);
+            }
+
             return new LoaderOptions(this);
         }
 
@@ -226,30 +253,61 @@ public class LoaderOptions
             return this;
         }
 
+        @Deprecated
         public Builder hosts(Set<InetAddress> hosts)
         {
-            this.hosts = hosts;
+            this.hostsArg.addAll(hosts);
+            return this;
+        }
+
+        public Builder hostsAndNativePort(Set<InetSocketAddress> hosts)
+        {
+            this.hosts.addAll(hosts);
             return this;
         }
 
         public Builder host(InetAddress host)
         {
+            hostsArg.add(host);
+            return this;
+        }
+
+        public Builder hostAndNativePort(InetSocketAddress host)
+        {
             hosts.add(host);
             return this;
         }
 
         public Builder ignore(Set<InetAddress> ignores)
         {
-            this.ignores = ignores;
+            this.ignoresArg.addAll(ignores);
+            return this;
+        }
+
+        public Builder ignoresAndInternalPorts(Set<InetAddressAndPort> ignores)
+        {
+            this.ignores.addAll(ignores);
             return this;
         }
 
         public Builder ignore(InetAddress ignore)
         {
+            ignoresArg.add(ignore);
+            return this;
+        }
+
+        public Builder ignoreAndInternalPorts(InetAddressAndPort ignore)
+        {
             ignores.add(ignore);
             return this;
         }
 
+        public Builder allowServerPortDiscovery(boolean allowServerPortDiscovery)
+        {
+            this.allowServerPortDiscovery = allowServerPortDiscovery;
+            return this;
+        }
+
         public Builder parseArgs(String cmdArgs[])
         {
             CommandLineParser parser = new GnuParser();
@@ -296,6 +354,7 @@ public class LoaderOptions
 
                 verbose = cmd.hasOption(VERBOSE_OPTION);
                 noProgress = cmd.hasOption(NOPROGRESS_OPTION);
+                allowServerPortDiscovery = cmd.hasOption(ALLOW_SERVER_PORT_DISCOVERY_OPTION);
 
                 if (cmd.hasOption(USER_OPTION))
                 {
@@ -319,7 +378,8 @@ public class LoaderOptions
                     {
                         for (String node : nodes)
                         {
-                            hosts.add(InetAddress.getByName(node.trim()));
+                            HostAndPort hap = HostAndPort.fromString(node);
+                            hosts.add(new InetSocketAddress(InetAddress.getByName(hap.getHost()), hap.getPortOrDefault(nativePort)));
                         }
                     } catch (UnknownHostException e)
                     {
@@ -340,7 +400,7 @@ public class LoaderOptions
                     {
                         for (String node : nodes)
                         {
-                            ignores.add(InetAddress.getByName(node.trim()));
+                            ignores.add(InetAddressAndPort.getByNameOverrideDefaults(node.trim(), storagePort));
                         }
                     } catch (UnknownHostException e)
                     {
@@ -554,6 +614,7 @@ public class LoaderOptions
         options.addOption("st", SSL_STORE_TYPE, "STORE-TYPE", "Client SSL: type of store");
         options.addOption("ciphers", SSL_CIPHER_SUITES, "CIPHER-SUITES", "Client SSL: comma-separated list of encryption suites to use");
         options.addOption("f", CONFIG_PATH, "path to config file", "cassandra.yaml file path for streaming throughput and client/server SSL.");
+        options.addOption("spd", ALLOW_SERVER_PORT_DISCOVERY_OPTION, "allow server port discovery", "Use ports published by server to decide how to connect. With SSL requires StartTLS to be used.");
         return options;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 8acb3c1..d330ed4 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -450,39 +450,39 @@ public class NodeProbe implements AutoCloseable
         ssProxy.drain();
     }
 
-    public Map<String, String> getTokenToEndpointMap()
+    public Map<String, String> getTokenToEndpointMap(boolean withPort)
     {
-        return ssProxy.getTokenToEndpointMap();
+        return withPort ? ssProxy.getTokenToEndpointWithPortMap() : ssProxy.getTokenToEndpointMap();
     }
 
-    public List<String> getLiveNodes()
+    public List<String> getLiveNodes(boolean withPort)
     {
-        return ssProxy.getLiveNodes();
+        return withPort ? ssProxy.getLiveNodesWithPort() : ssProxy.getLiveNodes();
     }
 
-    public List<String> getJoiningNodes()
+    public List<String> getJoiningNodes(boolean withPort)
     {
-        return ssProxy.getJoiningNodes();
+        return withPort ? ssProxy.getJoiningNodesWithPort() : ssProxy.getJoiningNodes();
     }
 
-    public List<String> getLeavingNodes()
+    public List<String> getLeavingNodes(boolean withPort)
     {
-        return ssProxy.getLeavingNodes();
+        return withPort ? ssProxy.getLeavingNodesWithPort() : ssProxy.getLeavingNodes();
     }
 
-    public List<String> getMovingNodes()
+    public List<String> getMovingNodes(boolean withPort)
     {
-        return ssProxy.getMovingNodes();
+        return withPort ? ssProxy.getMovingNodesWithPort() : ssProxy.getMovingNodes();
     }
 
-    public List<String> getUnreachableNodes()
+    public List<String> getUnreachableNodes(boolean withPort)
     {
-        return ssProxy.getUnreachableNodes();
+        return withPort ? ssProxy.getUnreachableNodesWithPort() : ssProxy.getUnreachableNodes();
     }
 
-    public Map<String, String> getLoadMap()
+    public Map<String, String> getLoadMap(boolean withPort)
     {
-        return ssProxy.getLoadMap();
+        return withPort ? ssProxy.getLoadMapWithPort() : ssProxy.getLoadMap();
     }
 
     public Map<InetAddress, Float> getOwnership()
@@ -490,11 +490,21 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.getOwnership();
     }
 
+    public Map<String, Float> getOwnershipWithPort()
+    {
+        return ssProxy.getOwnershipWithPort();
+    }
+
     public Map<InetAddress, Float> effectiveOwnership(String keyspace) throws IllegalStateException
     {
         return ssProxy.effectiveOwnership(keyspace);
     }
 
+    public Map<String, Float> effectiveOwnershipWithPort(String keyspace) throws IllegalStateException
+    {
+        return ssProxy.effectiveOwnershipWithPort(keyspace);
+    }
+
     public CacheServiceMBean getCacheServiceMBean()
     {
         String cachePath = "org.apache.cassandra.db:type=Caches";
@@ -557,9 +567,9 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.getLocalHostId();
     }
 
-    public Map<String, String> getHostIdMap()
+    public Map<String, String> getHostIdMap(boolean withPort)
     {
-        return ssProxy.getEndpointToHostId();
+        return withPort ? ssProxy.getEndpointWithPortToHostId() : ssProxy.getEndpointToHostId();
     }
 
     public String getLoadString()
@@ -686,9 +696,9 @@ public class NodeProbe implements AutoCloseable
         ssProxy.removeNode(token);
     }
 
-    public String getRemovalStatus()
+    public String getRemovalStatus(boolean withPort)
     {
-        return ssProxy.getRemovalStatus();
+        return withPort ? ssProxy.getRemovalStatusWithPort() : ssProxy.getRemovalStatus();
     }
 
     public void forceRemoveCompletion()
@@ -775,6 +785,11 @@ public class NodeProbe implements AutoCloseable
         ssProxy.setHintedHandoffThrottleInKB(throttleInKB);
     }
 
+    public List<String> getEndpointsWithPort(String keyspace, String cf, String key)
+    {
+        return ssProxy.getNaturalEndpointsWithPort(keyspace, cf, key);
+    }
+
     public List<InetAddress> getEndpoints(String keyspace, String cf, String key)
     {
         return ssProxy.getNaturalEndpoints(keyspace, cf, key);
@@ -1144,9 +1159,9 @@ public class NodeProbe implements AutoCloseable
         ssProxy.rebuildSecondaryIndex(ksName, cfName, idxNames);
     }
 
-    public String getGossipInfo()
+    public String getGossipInfo(boolean withPort)
     {
-        return fdProxy.getAllEndpointStates();
+        return withPort ? fdProxy.getAllEndpointStatesWithPort() : fdProxy.getAllEndpointStates();
     }
 
     public void stop(String string)
@@ -1212,9 +1227,9 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.getSchemaVersion();
     }
 
-    public List<String> describeRing(String keyspaceName) throws IOException
+    public List<String> describeRing(String keyspaceName, boolean withPort) throws IOException
     {
-        return ssProxy.describeRingJMX(keyspaceName);
+        return withPort ? ssProxy.describeRingWithPortJMX(keyspaceName) : ssProxy.describeRingJMX(keyspaceName);
     }
 
     public void rebuild(String sourceDc, String keyspace, String tokens, String specificSources)
@@ -1580,11 +1595,11 @@ public class NodeProbe implements AutoCloseable
         }
     }
 
-    public TabularData getFailureDetectorPhilValues()
+    public TabularData getFailureDetectorPhilValues(boolean withPort)
     {
         try
         {
-            return fdProxy.getPhiValues();
+            return withPort ? fdProxy.getPhiValuesWithPort() : fdProxy.getPhiValues();
         }
         catch (OpenDataException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 59d4ead..d707499 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -248,6 +248,9 @@ public class NodeTool
         @Option(type = OptionType.GLOBAL, name = {"-pwf", "--password-file"}, description = "Path to the JMX password file")
         private String passwordFilePath = EMPTY;
 
+        @Option(type = OptionType.GLOBAL, name = { "-wp", "--with-port"}, description = "Operate in 4.0 mode with hosts disambiguated by port number", arity = 0)
+        protected boolean withPort = false;
+
         @Override
         public void run()
         {
@@ -398,4 +401,27 @@ public class NodeTool
         }
         return ownershipByDc;
     }
+
+    public static SortedMap<String, SetHostStatWithPort> getOwnershipByDcWithPort(NodeProbe probe, boolean resolveIp,
+                                                                  Map<String, String> tokenToEndpoint,
+                                                                  Map<String, Float> ownerships)
+    {
+        SortedMap<String, SetHostStatWithPort> ownershipByDc = Maps.newTreeMap();
+        EndpointSnitchInfoMBean epSnitchInfo = probe.getEndpointSnitchInfoProxy();
+        try
+        {
+            for (Entry<String, String> tokenAndEndPoint : tokenToEndpoint.entrySet())
+            {
+                String dc = epSnitchInfo.getDatacenter(tokenAndEndPoint.getValue());
+                if (!ownershipByDc.containsKey(dc))
+                    ownershipByDc.put(dc, new SetHostStatWithPort(resolveIp));
+                ownershipByDc.get(dc).add(tokenAndEndPoint.getKey(), tokenAndEndPoint.getValue(), ownerships);
+            }
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+        return ownershipByDc;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java b/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java
index 5228468..ed91b8b 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java
@@ -49,10 +49,10 @@ public class DescribeCluster extends NodeToolCmd
 
         // display schema version for each node
         System.out.println("\tSchema versions:");
-        Map<String, List<String>> schemaVersions = probe.getSpProxy().getSchemaVersions();
+        Map<String, List<String>> schemaVersions = withPort ? probe.getSpProxy().getSchemaVersionsWithPort() : probe.getSpProxy().getSchemaVersions();
         for (String version : schemaVersions.keySet())
         {
             System.out.println(format("\t\t%s: %s%n", version, schemaVersions.get(version)));
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java b/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java
index 2a73c2a..c57e54d 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java
@@ -39,7 +39,7 @@ public class DescribeRing extends NodeToolCmd
         System.out.println("TokenRange: ");
         try
         {
-            for (String tokenRangeString : probe.describeRing(keyspace))
+            for (String tokenRangeString : probe.describeRing(keyspace, withPort))
             {
                 System.out.println("\t" + tokenRangeString);
             }
@@ -48,4 +48,4 @@ public class DescribeRing extends NodeToolCmd
             throw new RuntimeException(e);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/FailureDetectorInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/FailureDetectorInfo.java b/src/java/org/apache/cassandra/tools/nodetool/FailureDetectorInfo.java
index b3ffb6d..896663d 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/FailureDetectorInfo.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/FailureDetectorInfo.java
@@ -33,7 +33,7 @@ public class FailureDetectorInfo extends NodeToolCmd
     @Override
     public void execute(NodeProbe probe)
     {
-        TabularData data = probe.getFailureDetectorPhilValues();
+        TabularData data = probe.getFailureDetectorPhilValues(withPort);
         System.out.printf("%10s,%16s%n", "Endpoint", "Phi");
         for (Object o : data.keySet())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/GetEndpoints.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetEndpoints.java b/src/java/org/apache/cassandra/tools/nodetool/GetEndpoints.java
index 922ae26..c0adb2a 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/GetEndpoints.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetEndpoints.java
@@ -42,10 +42,20 @@ public class GetEndpoints extends NodeToolCmd
         String table = args.get(1);
         String key = args.get(2);
 
-        List<InetAddress> endpoints = probe.getEndpoints(ks, table, key);
-        for (InetAddress endpoint : endpoints)
+        if (withPort)
         {
-            System.out.println(endpoint.getHostAddress());
+            for (String endpoint : probe.getEndpointsWithPort(ks, table, key))
+            {
+                System.out.println(endpoint);
+            }
+        }
+        else
+        {
+            List<InetAddress> endpoints = probe.getEndpoints(ks, table, key);
+            for (InetAddress endpoint : endpoints)
+            {
+                System.out.println(endpoint.getHostAddress());
+            }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/GossipInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GossipInfo.java b/src/java/org/apache/cassandra/tools/nodetool/GossipInfo.java
index 1b4b979..182c395 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/GossipInfo.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/GossipInfo.java
@@ -28,6 +28,6 @@ public class GossipInfo extends NodeToolCmd
     @Override
     public void execute(NodeProbe probe)
     {
-        System.out.println(probe.getGossipInfo());
+        System.out.println(probe.getGossipInfo(withPort));
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/HostStatWithPort.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/HostStatWithPort.java b/src/java/org/apache/cassandra/tools/nodetool/HostStatWithPort.java
new file mode 100644
index 0000000..54cda17
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/HostStatWithPort.java
@@ -0,0 +1,44 @@
+/*
+ * 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.tools.nodetool;
+
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+public class HostStatWithPort
+{
+    public final InetAddressAndPort endpoint;
+    public final boolean resolveIp;
+    public final Float owns;
+    public final String token;
+
+    public HostStatWithPort(String token, InetAddressAndPort endpoint, boolean resolveIp, Float owns)
+    {
+        this.token = token;
+        this.endpoint = endpoint;
+        this.resolveIp = resolveIp;
+        this.owns = owns;
+    }
+
+    public String ipOrDns()
+    {
+        return resolveIp ?
+               endpoint.address.getHostName() + ":" + endpoint.port :
+               endpoint.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/NetStats.java b/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
index 2312097..2702d9e 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
@@ -50,11 +50,11 @@ public class NetStats extends NodeToolCmd
             System.out.printf("%s %s%n", status.streamOperation.getDescription(), status.planId.toString());
             for (SessionInfo info : status.sessions)
             {
-                System.out.printf("    %s", info.peer.toString());
+                System.out.printf("    %s", info.peer.toString(withPort));
                 // print private IP when it is used
                 if (!info.peer.equals(info.connecting))
                 {
-                    System.out.printf(" (using %s)", info.connecting.toString());
+                    System.out.printf(" (using %s)", info.connecting.toString(withPort));
                 }
                 System.out.printf("%n");
                 if (!info.receivingSummaries.isEmpty())
@@ -65,7 +65,7 @@ public class NetStats extends NodeToolCmd
                         System.out.printf("        Receiving %d files, %d bytes total. Already received %d files, %d bytes total%n", info.getTotalFilesToReceive(), info.getTotalSizeToReceive(), info.getTotalFilesReceived(), info.getTotalSizeReceived());
                     for (ProgressInfo progress : info.getReceivingFiles())
                     {
-                        System.out.printf("            %s%n", progress.toString());
+                        System.out.printf("            %s%n", progress.toString(withPort));
                     }
                 }
                 if (!info.sendingSummaries.isEmpty())
@@ -76,7 +76,7 @@ public class NetStats extends NodeToolCmd
                         System.out.printf("        Sending %d files, %d bytes total. Already sent %d files, %d bytes total%n", info.getTotalFilesToSend(), info.getTotalSizeToSend(), info.getTotalFilesSent(), info.getTotalSizeSent());
                     for (ProgressInfo progress : info.getSendingFiles())
                     {
-                        System.out.printf("            %s%n", progress.toString());
+                        System.out.printf("            %s%n", progress.toString(withPort));
                     }
                 }
             }
@@ -131,4 +131,4 @@ public class NetStats extends NodeToolCmd
             System.out.printf("%-25s%10s%10s%15s%10s%n", "Gossip messages", "n/a", pending, completed, dropped);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java b/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java
index 7312597..bd40aba 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java
@@ -36,10 +36,10 @@ public class RemoveNode extends NodeToolCmd
         switch (removeOperation)
         {
             case "status":
-                System.out.println("RemovalStatus: " + probe.getRemovalStatus());
+                System.out.println("RemovalStatus: " + probe.getRemovalStatus(withPort));
                 break;
             case "force":
-                System.out.println("RemovalStatus: " + probe.getRemovalStatus());
+                System.out.println("RemovalStatus: " + probe.getRemovalStatus(withPort));
                 probe.forceRemoveCompletion();
                 break;
             default:
@@ -47,4 +47,4 @@ public class RemoveNode extends NodeToolCmd
                 break;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/RepairAdmin.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/RepairAdmin.java b/src/java/org/apache/cassandra/tools/nodetool/RepairAdmin.java
index 0b4f767..ba3cf62 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/RepairAdmin.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/RepairAdmin.java
@@ -57,7 +57,8 @@ public class RepairAdmin extends NodeTool.NodeToolCmd
                                                                   "state",
                                                                   "last activity",
                                                                   "coordinator",
-                                                                  "participants");
+                                                                  "participants",
+                                                                  "participants_wp");
 
 
     private List<String> sessionValues(Map<String, String> session, int now)
@@ -67,7 +68,8 @@ public class RepairAdmin extends NodeTool.NodeToolCmd
                                   session.get(LocalSessionInfo.STATE),
                                   Integer.toString(now - updated) + " (s)",
                                   session.get(LocalSessionInfo.COORDINATOR),
-                                  session.get(LocalSessionInfo.PARTICIPANTS));
+                                  session.get(LocalSessionInfo.PARTICIPANTS),
+                                  session.get(LocalSessionInfo.PARTICIPANTS_WP));
     }
 
     private void listSessions(ActiveRepairServiceMBean repairServiceProxy)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/Ring.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Ring.java b/src/java/org/apache/cassandra/tools/nodetool/Ring.java
index 9c389c2..105726c 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Ring.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Ring.java
@@ -51,59 +51,103 @@ public class Ring extends NodeToolCmd
     @Override
     public void execute(NodeProbe probe)
     {
-        Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap();
-        LinkedHashMultimap<String, String> endpointsToTokens = LinkedHashMultimap.create();
-        boolean haveVnodes = false;
-        for (Map.Entry<String, String> entry : tokensToEndpoints.entrySet())
-        {
-            haveVnodes |= endpointsToTokens.containsKey(entry.getValue());
-            endpointsToTokens.put(entry.getValue(), entry.getKey());
-        }
-
-        int maxAddressLength = Collections.max(endpointsToTokens.keys(), new Comparator<String>()
+        try
         {
-            @Override
-            public int compare(String first, String second)
+            Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap(withPort);
+            LinkedHashMultimap<String, String> endpointsToTokens = LinkedHashMultimap.create();
+            boolean haveVnodes = false;
+            for (Map.Entry<String, String> entry : tokensToEndpoints.entrySet())
             {
-            	return Integer.compare(first.length(), second.length());
+                haveVnodes |= endpointsToTokens.containsKey(entry.getValue());
+                endpointsToTokens.put(entry.getValue(), entry.getKey());
             }
-        }).length();
 
-        String formatPlaceholder = "%%-%ds  %%-12s%%-7s%%-8s%%-16s%%-20s%%-44s%%n";
-        String format = format(formatPlaceholder, maxAddressLength);
+            int maxAddressLength = Collections.max(endpointsToTokens.keys(), new Comparator<String>()
+            {
+                @Override
+                public int compare(String first, String second)
+                {
+                    return Integer.compare(first.length(), second.length());
+                }
+            }).length();
 
-        StringBuilder errors = new StringBuilder();
-        boolean showEffectiveOwnership = true;
-        // Calculate per-token ownership of the ring
-        Map<InetAddress, Float> ownerships;
-        try
-        {
-            ownerships = probe.effectiveOwnership(keyspace);
-        }
-        catch (IllegalStateException ex)
-        {
-            ownerships = probe.getOwnership();
-            errors.append("Note: ").append(ex.getMessage()).append("%n");
-            showEffectiveOwnership = false;
-        }
-        catch (IllegalArgumentException ex)
-        {
-            System.out.printf("%nError: %s%n", ex.getMessage());
-            return;
-        }
+            String formatPlaceholder = "%%-%ds  %%-12s%%-7s%%-8s%%-16s%%-20s%%-44s%%n";
+            String format = format(formatPlaceholder, maxAddressLength);
 
+            StringBuilder errors = new StringBuilder();
+            boolean showEffectiveOwnership = true;
 
-        System.out.println();
-        for (Entry<String, SetHostStat> entry : NodeTool.getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
-            printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
+            if (withPort)
+            {
+                // Calculate per-token ownership of the ring
+                Map<String, Float> ownerships;
+                try
+                {
+                    ownerships = probe.effectiveOwnershipWithPort(keyspace);
+                }
+                catch (IllegalStateException ex)
+                {
+                    ownerships = probe.getOwnershipWithPort();
+                    errors.append("Note: ").append(ex.getMessage()).append("%n");
+                    showEffectiveOwnership = false;
+                }
+                catch (IllegalArgumentException ex)
+                {
+                    System.out.printf("%nError: %s%n", ex.getMessage());
+                    return;
+                }
+
+
+                System.out.println();
+                for (Entry<String, SetHostStatWithPort> entry : NodeTool.getOwnershipByDcWithPort(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
+                    printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(), showEffectiveOwnership);
+
+                if (haveVnodes)
+                {
+                    System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
+                    System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
+                }
+
+                System.out.printf("%n  " + errors.toString());
+            }
+            else
+            {
+                // Calculate per-token ownership of the ring
+                Map<InetAddress, Float> ownerships;
+                try
+                {
+                    ownerships = probe.effectiveOwnership(keyspace);
+                }
+                catch (IllegalStateException ex)
+                {
+                    ownerships = probe.getOwnership();
+                    errors.append("Note: ").append(ex.getMessage()).append("%n");
+                    showEffectiveOwnership = false;
+                }
+                catch (IllegalArgumentException ex)
+                {
+                    System.out.printf("%nError: %s%n", ex.getMessage());
+                    return;
+                }
 
-        if (haveVnodes)
+
+                System.out.println();
+                for (Entry<String, SetHostStat> entry : NodeTool.getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
+                    printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(), showEffectiveOwnership);
+
+                if (haveVnodes)
+                {
+                    System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
+                    System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
+                }
+
+                System.out.printf("%n  " + errors.toString());
+            }
+        } catch (Exception e)
         {
-            System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
-            System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
+            e.printStackTrace();
+            throw e;
         }
-
-        System.out.printf("%n  " + errors.toString());
     }
 
     private void printDc(NodeProbe probe, String format,
@@ -111,12 +155,12 @@ public class Ring extends NodeToolCmd
                          LinkedHashMultimap<String, String> endpointsToTokens,
                          SetHostStat hoststats,boolean showEffectiveOwnership)
     {
-        Collection<String> liveNodes = probe.getLiveNodes();
-        Collection<String> deadNodes = probe.getUnreachableNodes();
-        Collection<String> joiningNodes = probe.getJoiningNodes();
-        Collection<String> leavingNodes = probe.getLeavingNodes();
-        Collection<String> movingNodes = probe.getMovingNodes();
-        Map<String, String> loadMap = probe.getLoadMap();
+        Collection<String> liveNodes = probe.getLiveNodes(false);
+        Collection<String> deadNodes = probe.getUnreachableNodes(false);
+        Collection<String> joiningNodes = probe.getJoiningNodes(false);
+        Collection<String> leavingNodes = probe.getLeavingNodes(false);
+        Collection<String> movingNodes = probe.getMovingNodes(false);
+        Map<String, String> loadMap = probe.getLoadMap(false);
 
         System.out.println("Datacenter: " + dc);
         System.out.println("==========");
@@ -174,4 +218,73 @@ public class Ring extends NodeToolCmd
         }
         System.out.println();
     }
+
+    private void printDc(NodeProbe probe, String format,
+                         String dc,
+                         LinkedHashMultimap<String, String> endpointsToTokens,
+                         SetHostStatWithPort hoststats,boolean showEffectiveOwnership)
+    {
+        Collection<String> liveNodes = probe.getLiveNodes(true);
+        Collection<String> deadNodes = probe.getUnreachableNodes(true);
+        Collection<String> joiningNodes = probe.getJoiningNodes(true);
+        Collection<String> leavingNodes = probe.getLeavingNodes(true);
+        Collection<String> movingNodes = probe.getMovingNodes(true);
+        Map<String, String> loadMap = probe.getLoadMap(true);
+
+        System.out.println("Datacenter: " + dc);
+        System.out.println("==========");
+
+        // get the total amount of replicas for this dc and the last token in this dc's ring
+        List<String> tokens = new ArrayList<>();
+        String lastToken = "";
+
+        for (HostStatWithPort stat : hoststats)
+        {
+            tokens.addAll(endpointsToTokens.get(stat.endpoint.toString()));
+            lastToken = tokens.get(tokens.size() - 1);
+        }
+
+        System.out.printf(format, "Address", "Rack", "Status", "State", "Load", "Owns", "Token");
+
+        if (hoststats.size() > 1)
+            System.out.printf(format, "", "", "", "", "", "", lastToken);
+        else
+            System.out.println();
+
+        for (HostStatWithPort stat : hoststats)
+        {
+            String endpoint = stat.endpoint.toString();
+            String rack;
+            try
+            {
+                rack = probe.getEndpointSnitchInfoProxy().getRack(endpoint);
+            }
+            catch (UnknownHostException e)
+            {
+                rack = "Unknown";
+            }
+
+            String status = liveNodes.contains(endpoint)
+                            ? "Up"
+                            : deadNodes.contains(endpoint)
+                              ? "Down"
+                              : "?";
+
+            String state = "Normal";
+
+            if (joiningNodes.contains(endpoint))
+                state = "Joining";
+            else if (leavingNodes.contains(endpoint))
+                state = "Leaving";
+            else if (movingNodes.contains(endpoint))
+                state = "Moving";
+
+            String load = loadMap.containsKey(endpoint)
+                          ? loadMap.get(endpoint)
+                          : "?";
+            String owns = stat.owns != null && showEffectiveOwnership? new DecimalFormat("##0.00%").format(stat.owns) : "?";
+            System.out.printf(format, stat.ipOrDns(), rack, status, state, load, owns, stat.token);
+        }
+        System.out.println();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/SetHostStatWithPort.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetHostStatWithPort.java b/src/java/org/apache/cassandra/tools/nodetool/SetHostStatWithPort.java
new file mode 100644
index 0000000..67cd464
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetHostStatWithPort.java
@@ -0,0 +1,56 @@
+/*
+ * 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.tools.nodetool;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+public class SetHostStatWithPort implements Iterable<HostStatWithPort>
+{
+    final List<HostStatWithPort> hostStats = new ArrayList<>();
+    final boolean resolveIp;
+
+    public SetHostStatWithPort(boolean resolveIp)
+    {
+        this.resolveIp = resolveIp;
+    }
+
+    public int size()
+    {
+        return hostStats.size();
+    }
+
+    @Override
+    public Iterator<HostStatWithPort> iterator()
+    {
+        return hostStats.iterator();
+    }
+
+    public void add(String token, String host, Map<String, Float> ownerships) throws UnknownHostException
+    {
+        InetAddressAndPort endpoint = InetAddressAndPort.getByName(host);
+        Float owns = ownerships.get(endpoint);
+        hostStats.add(new HostStatWithPort(token, endpoint, resolveIp, owns));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/nodetool/Status.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Status.java b/src/java/org/apache/cassandra/tools/nodetool/Status.java
index f3ef606..49724a5 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Status.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Status.java
@@ -30,6 +30,7 @@ import java.util.Map;
 import java.util.SortedMap;
 
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.tools.NodeProbe;
 import org.apache.cassandra.tools.NodeTool;
 import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
@@ -55,72 +56,130 @@ public class Status extends NodeToolCmd
     @Override
     public void execute(NodeProbe probe)
     {
-        joiningNodes = probe.getJoiningNodes();
-        leavingNodes = probe.getLeavingNodes();
-        movingNodes = probe.getMovingNodes();
-        loadMap = probe.getLoadMap();
-        Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap();
-        liveNodes = probe.getLiveNodes();
-        unreachableNodes = probe.getUnreachableNodes();
-        hostIDMap = probe.getHostIdMap();
+        joiningNodes = probe.getJoiningNodes(withPort);
+        leavingNodes = probe.getLeavingNodes(withPort);
+        movingNodes = probe.getMovingNodes(withPort);
+        loadMap = probe.getLoadMap(withPort);
+        Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap(withPort);
+        liveNodes = probe.getLiveNodes(withPort);
+        unreachableNodes = probe.getUnreachableNodes(withPort);
+        hostIDMap = probe.getHostIdMap(withPort);
         epSnitchInfo = probe.getEndpointSnitchInfoProxy();
 
         StringBuilder errors = new StringBuilder();
 
-        Map<InetAddress, Float> ownerships = null;
-        boolean hasEffectiveOwns = false;
-        try
-        {
-            ownerships = probe.effectiveOwnership(keyspace);
-            hasEffectiveOwns = true;
-        }
-        catch (IllegalStateException e)
-        {
-            ownerships = probe.getOwnership();
-            errors.append("Note: ").append(e.getMessage()).append("%n");
-        }
-        catch (IllegalArgumentException ex)
+        if (withPort)
         {
-            System.out.printf("%nError: %s%n", ex.getMessage());
-            System.exit(1);
-        }
+            Map<String, Float> ownerships = null;
+            boolean hasEffectiveOwns = false;
+            try
+            {
+                ownerships = probe.effectiveOwnershipWithPort(keyspace);
+                hasEffectiveOwns = true;
+            }
+            catch (IllegalStateException e)
+            {
+                ownerships = probe.getOwnershipWithPort();
+                errors.append("Note: ").append(e.getMessage()).append("%n");
+            }
+            catch (IllegalArgumentException ex)
+            {
+                System.out.printf("%nError: %s%n", ex.getMessage());
+                System.exit(1);
+            }
 
-        SortedMap<String, SetHostStat> dcs = NodeTool.getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships);
+            SortedMap<String, SetHostStatWithPort> dcs = NodeTool.getOwnershipByDcWithPort(probe, resolveIp, tokensToEndpoints, ownerships);
 
-        // More tokens than nodes (aka vnodes)?
-        if (dcs.values().size() < tokensToEndpoints.keySet().size())
-            isTokenPerNode = false;
+            // More tokens than nodes (aka vnodes)?
+            if (dcs.values().size() < tokensToEndpoints.keySet().size())
+                isTokenPerNode = false;
 
-        findMaxAddressLength(dcs);
+            findMaxAddressLengthWithPort(dcs);
 
-        // Datacenters
-        for (Map.Entry<String, SetHostStat> dc : dcs.entrySet())
-        {
-            String dcHeader = String.format("Datacenter: %s%n", dc.getKey());
-            System.out.printf(dcHeader);
-            for (int i = 0; i < (dcHeader.length() - 1); i++) System.out.print('=');
-            System.out.println();
+            // Datacenters
+            for (Map.Entry<String, SetHostStatWithPort> dc : dcs.entrySet())
+            {
+                String dcHeader = String.format("Datacenter: %s%n", dc.getKey());
+                System.out.printf(dcHeader);
+                for (int i = 0; i < (dcHeader.length() - 1); i++) System.out.print('=');
+                System.out.println();
 
-            // Legend
-            System.out.println("Status=Up/Down");
-            System.out.println("|/ State=Normal/Leaving/Joining/Moving");
+                // Legend
+                System.out.println("Status=Up/Down");
+                System.out.println("|/ State=Normal/Leaving/Joining/Moving");
 
-            printNodesHeader(hasEffectiveOwns, isTokenPerNode);
+                printNodesHeader(hasEffectiveOwns, isTokenPerNode);
 
-            ArrayListMultimap<InetAddress, HostStat> hostToTokens = ArrayListMultimap.create();
-            for (HostStat stat : dc.getValue())
-                hostToTokens.put(stat.endpoint, stat);
+                ArrayListMultimap<InetAddressAndPort, HostStatWithPort> hostToTokens = ArrayListMultimap.create();
+                for (HostStatWithPort stat : dc.getValue())
+                    hostToTokens.put(stat.endpoint, stat);
 
-            for (InetAddress endpoint : hostToTokens.keySet())
-            {
-                Float owns = ownerships.get(endpoint);
-                List<HostStat> tokens = hostToTokens.get(endpoint);
-                printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
+                for (InetAddressAndPort endpoint : hostToTokens.keySet())
+                {
+                    Float owns = ownerships.get(endpoint);
+                    List<HostStatWithPort> tokens = hostToTokens.get(endpoint);
+                    printNodeWithPort(endpoint.toString(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
+                }
             }
+
+            System.out.printf("%n" + errors.toString());
         }
+        else
+        {
+            Map<InetAddress, Float> ownerships = null;
+            boolean hasEffectiveOwns = false;
+            try
+            {
+                ownerships = probe.effectiveOwnership(keyspace);
+                hasEffectiveOwns = true;
+            }
+            catch (IllegalStateException e)
+            {
+                ownerships = probe.getOwnership();
+                errors.append("Note: ").append(e.getMessage()).append("%n");
+            }
+            catch (IllegalArgumentException ex)
+            {
+                System.out.printf("%nError: %s%n", ex.getMessage());
+                System.exit(1);
+            }
+
+            SortedMap<String, SetHostStat> dcs = NodeTool.getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships);
 
-        System.out.printf("%n" + errors.toString());
+            // More tokens than nodes (aka vnodes)?
+            if (dcs.values().size() < tokensToEndpoints.keySet().size())
+                isTokenPerNode = false;
 
+            findMaxAddressLength(dcs);
+
+            // Datacenters
+            for (Map.Entry<String, SetHostStat> dc : dcs.entrySet())
+            {
+                String dcHeader = String.format("Datacenter: %s%n", dc.getKey());
+                System.out.printf(dcHeader);
+                for (int i = 0; i < (dcHeader.length() - 1); i++) System.out.print('=');
+                System.out.println();
+
+                // Legend
+                System.out.println("Status=Up/Down");
+                System.out.println("|/ State=Normal/Leaving/Joining/Moving");
+
+                printNodesHeader(hasEffectiveOwns, isTokenPerNode);
+
+                ArrayListMultimap<InetAddress, HostStat> hostToTokens = ArrayListMultimap.create();
+                for (HostStat stat : dc.getValue())
+                    hostToTokens.put(stat.endpoint, stat);
+
+                for (InetAddress endpoint : hostToTokens.keySet())
+                {
+                    Float owns = ownerships.get(endpoint);
+                    List<HostStat> tokens = hostToTokens.get(endpoint);
+                    printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
+                }
+            }
+
+            System.out.printf("%n" + errors.toString());
+        }
     }
 
     private void findMaxAddressLength(Map<String, SetHostStat> dcs)
@@ -135,6 +194,18 @@ public class Status extends NodeToolCmd
         }
     }
 
+    private void findMaxAddressLengthWithPort(Map<String, SetHostStatWithPort> dcs)
+    {
+        maxAddressLength = 0;
+        for (Map.Entry<String, SetHostStatWithPort> dc : dcs.entrySet())
+        {
+            for (HostStatWithPort stat : dc.getValue())
+            {
+                maxAddressLength = Math.max(maxAddressLength, stat.ipOrDns().length());
+            }
+        }
+    }
+
     private void printNodesHeader(boolean hasEffectiveOwns, boolean isTokenPerNode)
     {
         String fmt = getFormat(hasEffectiveOwns, isTokenPerNode);
@@ -177,6 +248,37 @@ public class Status extends NodeToolCmd
             System.out.printf(fmt, status, state, endpointDns, load, tokens.size(), strOwns, hostID, rack);
     }
 
+    private void printNodeWithPort(String endpoint, Float owns, List<HostStatWithPort> tokens, boolean hasEffectiveOwns, boolean isTokenPerNode)
+    {
+        String status, state, load, strOwns, hostID, rack, fmt;
+        fmt = getFormat(hasEffectiveOwns, isTokenPerNode);
+        if (liveNodes.contains(endpoint)) status = "U";
+        else if (unreachableNodes.contains(endpoint)) status = "D";
+        else status = "?";
+        if (joiningNodes.contains(endpoint)) state = "J";
+        else if (leavingNodes.contains(endpoint)) state = "L";
+        else if (movingNodes.contains(endpoint)) state = "M";
+        else state = "N";
+
+        load = loadMap.containsKey(endpoint) ? loadMap.get(endpoint) : "?";
+        strOwns = owns != null && hasEffectiveOwns ? new DecimalFormat("##0.0%").format(owns) : "?";
+        hostID = hostIDMap.get(endpoint);
+
+        try
+        {
+            rack = epSnitchInfo.getRack(endpoint);
+        } catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        String endpointDns = tokens.get(0).ipOrDns();
+        if (isTokenPerNode)
+            System.out.printf(fmt, status, state, endpointDns, load, strOwns, hostID, tokens.get(0).token, rack);
+        else
+            System.out.printf(fmt, status, state, endpointDns, load, tokens.size(), strOwns, hostID, rack);
+    }
+
     private String getFormat(
             boolean hasEffectiveOwns,
             boolean isTokenPerNode)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java b/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
index 9230d38..bf95080 100644
--- a/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
+++ b/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
@@ -29,7 +29,7 @@ class ExpiredTraceState extends TraceState
 
     ExpiredTraceState(TraceState delegate)
     {
-        super(FBUtilities.getBroadcastAddress(), delegate.sessionId, delegate.traceType);
+        super(FBUtilities.getBroadcastAddressAndPort(), delegate.sessionId, delegate.traceType);
         this.delegate = delegate;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
index 20c992c..487ed65 100644
--- a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
+++ b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
@@ -48,29 +48,31 @@ public final class TraceKeyspace
 
     private static final TableMetadata Sessions =
         parse(SESSIONS,
-              "tracing sessions",
-              "CREATE TABLE %s ("
-              + "session_id uuid,"
-              + "command text,"
-              + "client inet,"
-              + "coordinator inet,"
-              + "duration int,"
-              + "parameters map<text, text>,"
-              + "request text,"
-              + "started_at timestamp,"
-              + "PRIMARY KEY ((session_id)))");
+                "tracing sessions",
+                "CREATE TABLE %s ("
+                + "session_id uuid,"
+                + "command text,"
+                + "client inet,"
+                + "coordinator inet,"
+                + "coordinator_port int,"
+                + "duration int,"
+                + "parameters map<text, text>,"
+                + "request text,"
+                + "started_at timestamp,"
+                + "PRIMARY KEY ((session_id)))");
 
     private static final TableMetadata Events =
         parse(EVENTS,
-              "tracing events",
-              "CREATE TABLE %s ("
-              + "session_id uuid,"
-              + "event_id timeuuid,"
-              + "activity text,"
-              + "source inet,"
-              + "source_elapsed int,"
-              + "thread text,"
-              + "PRIMARY KEY ((session_id), event_id))");
+                "tracing events",
+                "CREATE TABLE %s ("
+                + "session_id uuid,"
+                + "event_id timeuuid,"
+                + "activity text,"
+                + "source inet,"
+                + "source_port int,"
+                + "source_elapsed int,"
+                + "thread text,"
+                + "PRIMARY KEY ((session_id), event_id))");
 
     private static TableMetadata parse(String table, String description, String cql)
     {
@@ -100,7 +102,8 @@ public final class TraceKeyspace
         builder.row()
                .ttl(ttl)
                .add("client", client)
-               .add("coordinator", FBUtilities.getBroadcastAddress())
+               .add("coordinator", FBUtilities.getBroadcastAddressAndPort().address)
+               .add("coordinator_port", FBUtilities.getBroadcastAddressAndPort().port)
                .add("request", request)
                .add("started_at", new Date(startedAt))
                .add("command", command)
@@ -125,7 +128,8 @@ public final class TraceKeyspace
                                               .ttl(ttl);
 
         rowBuilder.add("activity", message)
-                  .add("source", FBUtilities.getBroadcastAddress())
+                  .add("source", FBUtilities.getBroadcastAddressAndPort().address)
+                  .add("source_port", FBUtilities.getBroadcastAddressAndPort().port)
                   .add("thread", threadName);
 
         if (elapsed >= 0)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tracing/TraceState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/TraceState.java b/src/java/org/apache/cassandra/tracing/TraceState.java
index b4eff6b..a53846c 100644
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.tracing;
 
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.UUID;
@@ -28,6 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import com.google.common.base.Stopwatch;
 import org.slf4j.helpers.MessageFormatter;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.progress.ProgressEvent;
 import org.apache.cassandra.utils.progress.ProgressEventNotifier;
@@ -40,7 +40,7 @@ import org.apache.cassandra.utils.progress.ProgressListener;
 public abstract class TraceState implements ProgressEventNotifier
 {
     public final UUID sessionId;
-    public final InetAddress coordinator;
+    public final InetAddressAndPort coordinator;
     public final Stopwatch watch;
     public final ByteBuffer sessionIdBytes;
     public final Tracing.TraceType traceType;
@@ -63,7 +63,7 @@ public abstract class TraceState implements ProgressEventNotifier
     // See CASSANDRA-7626 for more details.
     private final AtomicInteger references = new AtomicInteger(1);
 
-    protected TraceState(InetAddress coordinator, UUID sessionId, Tracing.TraceType traceType)
+    protected TraceState(InetAddressAndPort coordinator, UUID sessionId, Tracing.TraceType traceType)
     {
         assert coordinator != null;
         assert sessionId != null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tracing/TraceStateImpl.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/TraceStateImpl.java b/src/java/org/apache/cassandra/tracing/TraceStateImpl.java
index 349000a..2722406 100644
--- a/src/java/org/apache/cassandra/tracing/TraceStateImpl.java
+++ b/src/java/org/apache/cassandra/tracing/TraceStateImpl.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.tracing;
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.Set;
 import java.util.UUID;
@@ -36,6 +35,7 @@ import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.exceptions.OverloadedException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.WrappedRunnable;
@@ -54,7 +54,7 @@ public class TraceStateImpl extends TraceState
 
     private final Set<Future<?>> pendingFutures = ConcurrentHashMap.newKeySet();
 
-    public TraceStateImpl(InetAddress coordinator, UUID sessionId, Tracing.TraceType traceType)
+    public TraceStateImpl(InetAddressAndPort coordinator, UUID sessionId, Tracing.TraceType traceType)
     {
         super(coordinator, sessionId, traceType);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 4cdddba..55e36a4 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -19,15 +19,17 @@
  */
 package org.apache.cassandra.tracing;
 
+import java.io.IOException;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,8 +37,13 @@ import io.netty.util.concurrent.FastThreadLocal;
 import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.ParameterType;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.UUIDGen;
@@ -48,8 +55,23 @@ import org.apache.cassandra.utils.UUIDGen;
  */
 public abstract class Tracing implements ExecutorLocal<TraceState>
 {
-    public static final String TRACE_HEADER = "TraceSession";
-    public static final String TRACE_TYPE = "TraceType";
+    public static final IVersionedSerializer<TraceType> traceTypeSerializer = new IVersionedSerializer<TraceType>()
+    {
+        public void serialize(TraceType traceType, DataOutputPlus out, int version) throws IOException
+        {
+            out.write((byte)traceType.ordinal());
+        }
+
+        public TraceType deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return TraceType.deserialize(in.readByte());
+        }
+
+        public long serializedSize(TraceType traceType, int version)
+        {
+            return 1;
+        }
+    };
 
     public enum TraceType
     {
@@ -83,7 +105,7 @@ public abstract class Tracing implements ExecutorLocal<TraceState>
 
     protected static final Logger logger = LoggerFactory.getLogger(Tracing.class);
 
-    private final InetAddress localAddress = FBUtilities.getLocalAddress();
+    private final InetAddressAndPort localAddress = FBUtilities.getLocalAddressAndPort();
 
     private final FastThreadLocal<TraceState> state = new FastThreadLocal<>();
 
@@ -228,21 +250,19 @@ public abstract class Tracing implements ExecutorLocal<TraceState>
      */
     public TraceState initializeFromMessage(final MessageIn<?> message)
     {
-        final byte[] sessionBytes = message.parameters.get(TRACE_HEADER);
+        final UUID sessionId = (UUID)message.parameters.get(ParameterType.TRACE_SESSION);
 
-        if (sessionBytes == null)
+        if (sessionId == null)
             return null;
 
-        assert sessionBytes.length == 16;
-        UUID sessionId = UUIDGen.getUUID(ByteBuffer.wrap(sessionBytes));
         TraceState ts = get(sessionId);
         if (ts != null && ts.acquireReference())
             return ts;
 
-        byte[] tmpBytes;
+        TraceType tmpType;
         TraceType traceType = TraceType.QUERY;
-        if ((tmpBytes = message.parameters.get(TRACE_TYPE)) != null)
-            traceType = TraceType.deserialize(tmpBytes[0]);
+        if ((tmpType = (TraceType)message.parameters.get(ParameterType.TRACE_TYPE)) != null)
+            traceType = tmpType;
 
         if (message.verb == MessagingService.Verb.REQUEST_RESPONSE)
         {
@@ -257,16 +277,16 @@ public abstract class Tracing implements ExecutorLocal<TraceState>
         }
     }
 
-    public Map<String, byte[]> getTraceHeaders()
+    public List<Object> getTraceHeaders()
     {
         assert isTracing();
 
-        return ImmutableMap.of(
-                TRACE_HEADER, UUIDGen.decompose(Tracing.instance.getSessionId()),
-                TRACE_TYPE, new byte[] { Tracing.TraceType.serialize(Tracing.instance.getTraceType()) });
+        return ImmutableList.of(
+        ParameterType.TRACE_SESSION, Tracing.instance.getSessionId(),
+        ParameterType.TRACE_TYPE, Tracing.instance.getTraceType());
     }
 
-    protected abstract TraceState newTraceState(InetAddress coordinator, UUID sessionId, Tracing.TraceType traceType);
+    protected abstract TraceState newTraceState(InetAddressAndPort coordinator, UUID sessionId, Tracing.TraceType traceType);
 
     // repair just gets a varargs method since it's so heavyweight anyway
     public static void traceRepair(String format, Object... args)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 789216e..1e32f10 100644
--- a/src/java/org/apache/cassandra/tracing/TracingImpl.java
+++ b/src/java/org/apache/cassandra/tracing/TracingImpl.java
@@ -26,6 +26,7 @@ import java.util.UUID;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.WrappedRunnable;
 
 
@@ -93,7 +94,7 @@ class TracingImpl extends Tracing
     }
 
     @Override
-    protected TraceState newTraceState(InetAddress coordinator, UUID sessionId, TraceType traceType)
+    protected TraceState newTraceState(InetAddressAndPort coordinator, UUID sessionId, TraceType traceType)
     {
         return new TraceStateImpl(coordinator, sessionId, traceType);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/transport/Event.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Event.java b/src/java/org/apache/cassandra/transport/Event.java
index ed77e59..77edf8a 100644
--- a/src/java/org/apache/cassandra/transport/Event.java
+++ b/src/java/org/apache/cassandra/transport/Event.java
@@ -24,6 +24,7 @@ import java.util.List;
 
 import com.google.common.base.Objects;
 import io.netty.buffer.ByteBuf;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public abstract class Event
 {
@@ -110,19 +111,19 @@ public abstract class Event
             this.change = change;
         }
 
-        public static TopologyChange newNode(InetAddress host, int port)
+        public static TopologyChange newNode(InetAddressAndPort address)
         {
-            return new TopologyChange(Change.NEW_NODE, new InetSocketAddress(host, port));
+            return new TopologyChange(Change.NEW_NODE, new InetSocketAddress(address.address, address.port));
         }
 
-        public static TopologyChange removedNode(InetAddress host, int port)
+        public static TopologyChange removedNode(InetAddressAndPort address)
         {
-            return new TopologyChange(Change.REMOVED_NODE, new InetSocketAddress(host, port));
+            return new TopologyChange(Change.REMOVED_NODE, new InetSocketAddress(address.address, address.port));
         }
 
-        public static TopologyChange movedNode(InetAddress host, int port)
+        public static TopologyChange movedNode(InetAddressAndPort address)
         {
-            return new TopologyChange(Change.MOVED_NODE, new InetSocketAddress(host, port));
+            return new TopologyChange(Change.MOVED_NODE, new InetSocketAddress(address.address, address.port));
         }
 
         // Assumes the type has already been deserialized
@@ -181,14 +182,14 @@ public abstract class Event
             this.status = status;
         }
 
-        public static StatusChange nodeUp(InetAddress host, int port)
+        public static StatusChange nodeUp(InetAddressAndPort address)
         {
-            return new StatusChange(Status.UP, new InetSocketAddress(host, port));
+            return new StatusChange(Status.UP, new InetSocketAddress(address.address, address.port));
         }
 
-        public static StatusChange nodeDown(InetAddress host, int port)
+        public static StatusChange nodeDown(InetAddressAndPort address)
         {
-            return new StatusChange(Status.DOWN, new InetSocketAddress(host, port));
+            return new StatusChange(Status.DOWN, new InetSocketAddress(address.address, address.port));
         }
 
         // Assumes the type has already been deserialized

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/transport/ProtocolVersion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/ProtocolVersion.java b/src/java/org/apache/cassandra/transport/ProtocolVersion.java
index cd73c86..838176a 100644
--- a/src/java/org/apache/cassandra/transport/ProtocolVersion.java
+++ b/src/java/org/apache/cassandra/transport/ProtocolVersion.java
@@ -43,7 +43,8 @@ public enum ProtocolVersion implements Comparable<ProtocolVersion>
     V2(2, "v2", false), // no longer supported
     V3(3, "v3", false),
     V4(4, "v4", false),
-    V5(5, "v5-beta", true);
+    V5(5, "v5", false),
+    V6(6, "v6-beta", true);
 
     /** The version number */
     private final int num;
@@ -62,7 +63,7 @@ public enum ProtocolVersion implements Comparable<ProtocolVersion>
     }
 
     /** The supported versions stored as an array, these should be private and are required for fast decoding*/
-    private final static ProtocolVersion[] SUPPORTED_VERSIONS = new ProtocolVersion[] { V3, V4, V5 };
+    private final static ProtocolVersion[] SUPPORTED_VERSIONS = new ProtocolVersion[] { V3, V4, V5, V6 };
     final static ProtocolVersion MIN_SUPPORTED_VERSION = SUPPORTED_VERSIONS[0];
     final static ProtocolVersion MAX_SUPPORTED_VERSION = SUPPORTED_VERSIONS[SUPPORTED_VERSIONS.length - 1];
 
@@ -73,8 +74,8 @@ public enum ProtocolVersion implements Comparable<ProtocolVersion>
     public final static EnumSet<ProtocolVersion> UNSUPPORTED = EnumSet.complementOf(SUPPORTED);
 
     /** The preferred versions */
-    public final static ProtocolVersion CURRENT = V4;
-    public final static Optional<ProtocolVersion> BETA = Optional.of(V5);
+    public final static ProtocolVersion CURRENT = V5;
+    public final static Optional<ProtocolVersion> BETA = Optional.of(V6);
 
     public static List<String> supportedVersions()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 d3f1c2c..cd04edc 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -51,6 +51,7 @@ import io.netty.util.internal.logging.Slf4JLoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.SchemaChangeListener;
 import org.apache.cassandra.security.SSLFactory;
@@ -454,51 +455,32 @@ public class Server implements CassandraDaemon.Server
 
         // We keep track of the latest status change events we have sent to avoid sending duplicates
         // since StorageService may send duplicate notifications (CASSANDRA-7816, CASSANDRA-8236, CASSANDRA-9156)
-        private final Map<InetAddress, LatestEvent> latestEvents = new ConcurrentHashMap<>();
+        private final Map<InetAddressAndPort, LatestEvent> latestEvents = new ConcurrentHashMap<>();
         // We also want to delay delivering a NEW_NODE notification until the new node has set its RPC ready
         // state. This tracks the endpoints which have joined, but not yet signalled they're ready for clients
-        private final Set<InetAddress> endpointsPendingJoinedNotification = ConcurrentHashMap.newKeySet();
-
-
-        private static final InetAddress bindAll;
-        static
-        {
-            try
-            {
-                bindAll = InetAddress.getByAddress(new byte[4]);
-            }
-            catch (UnknownHostException e)
-            {
-                throw new AssertionError(e);
-            }
-        }
+        private final Set<InetAddressAndPort> endpointsPendingJoinedNotification = ConcurrentHashMap.newKeySet();
 
         private EventNotifier(Server server)
         {
             this.server = server;
         }
 
-        private InetAddress getRpcAddress(InetAddress endpoint)
+        private InetAddressAndPort getNativeAddress(InetAddressAndPort endpoint)
         {
             try
             {
-                InetAddress rpcAddress = InetAddress.getByName(StorageService.instance.getRpcaddress(endpoint));
-                // If rpcAddress == 0.0.0.0 (i.e. bound on all addresses), returning that is not very helpful,
-                // so return the internal address (which is ok since "we're bound on all addresses").
-                // Note that after all nodes are running a version that includes CASSANDRA-5899, rpcAddress should
-                // never be 0.0.0.0, so this can eventually be removed.
-                return rpcAddress.equals(bindAll) ? endpoint : rpcAddress;
+                return InetAddressAndPort.getByName(StorageService.instance.getNativeaddress(endpoint, true));
             }
             catch (UnknownHostException e)
             {
                 // That should not happen, so log an error, but return the
                 // endpoint address since there's a good change this is right
                 logger.error("Problem retrieving RPC address for {}", endpoint, e);
-                return endpoint;
+                return InetAddressAndPort.getByAddressOverrideDefaults(endpoint.address, DatabaseDescriptor.getNativeTransportPort());
             }
         }
 
-        private void send(InetAddress endpoint, Event.NodeEvent event)
+        private void send(InetAddressAndPort endpoint, Event.NodeEvent event)
         {
             if (logger.isTraceEnabled())
                 logger.trace("Sending event for endpoint {}, rpc address {}", endpoint, event.nodeAddress());
@@ -508,8 +490,8 @@ public class Server implements CassandraDaemon.Server
             // then don't send the notification. This covers the case of rpc_address set to "localhost",
             // which is not useful to any driver and in fact may cauase serious problems to some drivers,
             // see CASSANDRA-10052
-            if (!endpoint.equals(FBUtilities.getBroadcastAddress()) &&
-                event.nodeAddress().equals(FBUtilities.getBroadcastRpcAddress()))
+            if (!endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) &&
+                event.nodeAddress().equals(FBUtilities.getJustBroadcastNativeAddress()))
                 return;
 
             send(event);
@@ -520,38 +502,38 @@ public class Server implements CassandraDaemon.Server
             server.connectionTracker.send(event);
         }
 
-        public void onJoinCluster(InetAddress endpoint)
+        public void onJoinCluster(InetAddressAndPort endpoint)
         {
             if (!StorageService.instance.isRpcReady(endpoint))
                 endpointsPendingJoinedNotification.add(endpoint);
             else
-                onTopologyChange(endpoint, Event.TopologyChange.newNode(getRpcAddress(endpoint), server.socket.getPort()));
+                onTopologyChange(endpoint, Event.TopologyChange.newNode(getNativeAddress(endpoint)));
         }
 
-        public void onLeaveCluster(InetAddress endpoint)
+        public void onLeaveCluster(InetAddressAndPort endpoint)
         {
-            onTopologyChange(endpoint, Event.TopologyChange.removedNode(getRpcAddress(endpoint), server.socket.getPort()));
+            onTopologyChange(endpoint, Event.TopologyChange.removedNode(getNativeAddress(endpoint)));
         }
 
-        public void onMove(InetAddress endpoint)
+        public void onMove(InetAddressAndPort endpoint)
         {
-            onTopologyChange(endpoint, Event.TopologyChange.movedNode(getRpcAddress(endpoint), server.socket.getPort()));
+            onTopologyChange(endpoint, Event.TopologyChange.movedNode(getNativeAddress(endpoint)));
         }
 
-        public void onUp(InetAddress endpoint)
+        public void onUp(InetAddressAndPort endpoint)
         {
             if (endpointsPendingJoinedNotification.remove(endpoint))
                 onJoinCluster(endpoint);
 
-            onStatusChange(endpoint, Event.StatusChange.nodeUp(getRpcAddress(endpoint), server.socket.getPort()));
+            onStatusChange(endpoint, Event.StatusChange.nodeUp(getNativeAddress(endpoint)));
         }
 
-        public void onDown(InetAddress endpoint)
+        public void onDown(InetAddressAndPort endpoint)
         {
-            onStatusChange(endpoint, Event.StatusChange.nodeDown(getRpcAddress(endpoint), server.socket.getPort()));
+            onStatusChange(endpoint, Event.StatusChange.nodeDown(getNativeAddress(endpoint)));
         }
 
-        private void onTopologyChange(InetAddress endpoint, Event.TopologyChange event)
+        private void onTopologyChange(InetAddressAndPort endpoint, Event.TopologyChange event)
         {
             if (logger.isTraceEnabled())
                 logger.trace("Topology changed event : {}, {}", endpoint, event.change);
@@ -565,7 +547,7 @@ public class Server implements CassandraDaemon.Server
             }
         }
 
-        private void onStatusChange(InetAddress endpoint, Event.StatusChange event)
+        private void onStatusChange(InetAddressAndPort endpoint, Event.StatusChange event)
         {
             if (logger.isTraceEnabled())
                 logger.trace("Status changed event : {}, {}", endpoint, event.status);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
index 9163d56..84af41c 100644
--- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.transport.*;
 import org.apache.cassandra.utils.MD5Digest;
 
@@ -88,14 +89,14 @@ public class ErrorMessage extends Message.Response
                         // The number of failures is also present in protocol v5, but used instead to specify the size of the failure map
                         int failure = body.readInt();
 
-                        Map<InetAddress, RequestFailureReason> failureReasonByEndpoint = new ConcurrentHashMap<>();
+                        Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint = new ConcurrentHashMap<>();
                         if (version.isGreaterOrEqualTo(ProtocolVersion.V5))
                         {
                             for (int i = 0; i < failure; i++)
                             {
                                 InetAddress endpoint = CBUtil.readInetAddr(body);
                                 RequestFailureReason failureReason = RequestFailureReason.fromCode(body.readUnsignedShort());
-                                failureReasonByEndpoint.put(endpoint, failureReason);
+                                failureReasonByEndpoint.put(InetAddressAndPort.getByAddress(endpoint), failureReason);
                             }
                         }
 
@@ -195,9 +196,9 @@ public class ErrorMessage extends Message.Response
 
                         if (version.isGreaterOrEqualTo(ProtocolVersion.V5))
                         {
-                            for (Map.Entry<InetAddress, RequestFailureReason> entry : rfe.failureReasonByEndpoint.entrySet())
+                            for (Map.Entry<InetAddressAndPort, RequestFailureReason> entry : rfe.failureReasonByEndpoint.entrySet())
                             {
-                                CBUtil.writeInetAddr(entry.getKey(), dest);
+                                CBUtil.writeInetAddr(entry.getKey().address, dest);
                                 dest.writeShort(entry.getValue().code);
                             }
                         }
@@ -260,9 +261,9 @@ public class ErrorMessage extends Message.Response
 
                         if (version.isGreaterOrEqualTo(ProtocolVersion.V5))
                         {
-                            for (Map.Entry<InetAddress, RequestFailureReason> entry : rfe.failureReasonByEndpoint.entrySet())
+                            for (Map.Entry<InetAddressAndPort, RequestFailureReason> entry : rfe.failureReasonByEndpoint.entrySet())
                             {
-                                size += CBUtil.sizeOfInetAddr(entry.getKey());
+                                size += CBUtil.sizeOfInetAddr(entry.getKey().address);
                                 size += 2; // RequestFailureReason code
                             }
                         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 1cb59d4..3ca8b89 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -58,6 +58,7 @@ import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.AsyncOneResponse;
 
 import org.codehaus.jackson.JsonFactory;
@@ -78,7 +79,10 @@ public class FBUtilities
 
     private static volatile InetAddress localInetAddress;
     private static volatile InetAddress broadcastInetAddress;
-    private static volatile InetAddress broadcastRpcAddress;
+    private static volatile InetAddress broadcastNativeAddress;
+    private static volatile InetAddressAndPort broadcastNativeAddressAndPort;
+    private static volatile InetAddressAndPort broadcastInetAddressAndPort;
+    private static volatile InetAddressAndPort localInetAddressAndPort;
 
     public static int getAvailableProcessors()
     {
@@ -92,9 +96,9 @@ public class FBUtilities
     public static final int MAX_UNSIGNED_SHORT = 0xFFFF;
 
     /**
-     * Please use getBroadcastAddress instead. You need this only when you have to listen/connect.
+     * Please use getJustBroadcastAddress instead. You need this only when you have to listen/connect.
      */
-    public static InetAddress getLocalAddress()
+    public static InetAddress getJustLocalAddress()
     {
         if (localInetAddress == null)
             try
@@ -110,30 +114,57 @@ public class FBUtilities
         return localInetAddress;
     }
 
-    public static InetAddress getBroadcastAddress()
+    public static InetAddressAndPort getLocalAddressAndPort()
+    {
+        if (localInetAddressAndPort == null)
+        {
+            localInetAddressAndPort = InetAddressAndPort.getByAddress(getJustLocalAddress());
+        }
+        return localInetAddressAndPort;
+    }
+
+    public static InetAddress getJustBroadcastAddress()
     {
         if (broadcastInetAddress == null)
             broadcastInetAddress = DatabaseDescriptor.getBroadcastAddress() == null
-                                 ? getLocalAddress()
+                                 ? getJustLocalAddress()
                                  : DatabaseDescriptor.getBroadcastAddress();
         return broadcastInetAddress;
     }
 
+    public static InetAddressAndPort getBroadcastAddressAndPort()
+    {
+        if (broadcastInetAddressAndPort == null)
+        {
+            broadcastInetAddressAndPort = InetAddressAndPort.getByAddress(getJustBroadcastAddress());
+        }
+        return broadcastInetAddressAndPort;
+    }
+
     /**
      * <b>THIS IS FOR TESTING ONLY!!</b>
      */
     public static void setBroadcastInetAddress(InetAddress addr)
     {
         broadcastInetAddress = addr;
+        broadcastInetAddressAndPort = InetAddressAndPort.getByAddress(broadcastInetAddress);
     }
 
-    public static InetAddress getBroadcastRpcAddress()
+    public static InetAddress getJustBroadcastNativeAddress()
     {
-        if (broadcastRpcAddress == null)
-            broadcastRpcAddress = DatabaseDescriptor.getBroadcastRpcAddress() == null
+        if (broadcastNativeAddress == null)
+            broadcastNativeAddress = DatabaseDescriptor.getBroadcastRpcAddress() == null
                                    ? DatabaseDescriptor.getRpcAddress()
                                    : DatabaseDescriptor.getBroadcastRpcAddress();
-        return broadcastRpcAddress;
+        return broadcastNativeAddress;
+    }
+
+    public static InetAddressAndPort getBroadcastNativeAddressAndPort()
+    {
+        if (broadcastNativeAddressAndPort == null)
+            broadcastNativeAddressAndPort = InetAddressAndPort.getByAddressOverrideDefaults(getJustBroadcastNativeAddress(),
+                                                                                             DatabaseDescriptor.getNativeTransportPort());
+        return broadcastNativeAddressAndPort;
     }
 
     public static Collection<InetAddress> getAllLocalAddresses()
@@ -838,7 +869,9 @@ public class FBUtilities
     public static void reset()
     {
         localInetAddress = null;
+        localInetAddressAndPort = null;
         broadcastInetAddress = null;
-        broadcastRpcAddress = null;
+        broadcastInetAddressAndPort = null;
+        broadcastNativeAddress = null;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/utils/JMXServerUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
index e78ed01..bb5c3ac 100644
--- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java
+++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
@@ -236,7 +236,7 @@ public class JMXServerUtils
         String hostName;
         if (serverAddress == null)
         {
-            hostName = FBUtilities.getBroadcastAddress() instanceof Inet6Address ? "[::]" : "0.0.0.0";
+            hostName = FBUtilities.getJustBroadcastAddress() instanceof Inet6Address ? "[::]" : "0.0.0.0";
         }
         else
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/utils/Mx4jTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Mx4jTool.java b/src/java/org/apache/cassandra/utils/Mx4jTool.java
index cd42aca..5baaea2 100644
--- a/src/java/org/apache/cassandra/utils/Mx4jTool.java
+++ b/src/java/org/apache/cassandra/utils/Mx4jTool.java
@@ -79,7 +79,7 @@ public class Mx4jTool
     {
         String sAddress = System.getProperty("mx4jaddress");
         if (StringUtils.isEmpty(sAddress))
-            sAddress = FBUtilities.getBroadcastAddress().getHostAddress();
+            sAddress = FBUtilities.getBroadcastAddressAndPort().address.getHostAddress();
         return sAddress;
     }
 


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


[13/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 7844c9b..d891043 100644
--- a/src/java/org/apache/cassandra/net/async/NettyFactory.java
+++ b/src/java/org/apache/cassandra/net/async/NettyFactory.java
@@ -48,6 +48,7 @@ import org.apache.cassandra.auth.IInternodeAuthenticator;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.service.NativeTransportService;
@@ -184,9 +185,9 @@ public final class NettyFactory
      * 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
+    public Channel createInboundChannel(InetAddressAndPort localAddr, InboundInitializer initializer, int receiveBufferSize) throws ConfigurationException
     {
-        String nic = FBUtilities.getNetworkInterface(localAddr.getAddress());
+        String nic = FBUtilities.getNetworkInterface(localAddr.address);
         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;
@@ -202,7 +203,7 @@ public final class NettyFactory
         if (receiveBufferSize > 0)
             bootstrap.childOption(ChannelOption.SO_RCVBUF, receiveBufferSize);
 
-        ChannelFuture channelFuture = bootstrap.bind(localAddr);
+        ChannelFuture channelFuture = bootstrap.bind(new InetSocketAddress(localAddr.address, localAddr.port));
 
         if (!channelFuture.awaitUninterruptibly().isSuccess())
         {
@@ -333,8 +334,9 @@ public final class NettyFactory
                               .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());
+        bootstrap.localAddress(params.connectionId.local().address, 0);
+        InetAddressAndPort remoteAddress = params.connectionId.connectionAddress();
+        bootstrap.remoteAddress(new InetSocketAddress(remoteAddress.address, remoteAddress.port));
         return bootstrap;
     }
 
@@ -362,7 +364,8 @@ public final class NettyFactory
             {
                 SslContext sslContext = SSLFactory.getSslContext(params.encryptionOptions, true, false);
                 // for some reason channel.remoteAddress() will return null
-                InetSocketAddress peer = params.encryptionOptions.require_endpoint_verification ? params.connectionId.remoteAddress() : null;
+                InetAddressAndPort address = params.connectionId.remote();
+                InetSocketAddress peer = params.encryptionOptions.require_endpoint_verification ? new InetSocketAddress(address.address, address.port) : null;
                 SslHandler sslHandler = newSslHandler(channel, sslContext, peer);
                 logger.trace("creating outbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
                 pipeline.addFirst(SSL_CHANNEL_HANDLER_NAME, sslHandler);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 6b2ff0d..f3cb554 100644
--- a/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
+++ b/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java
@@ -21,6 +21,8 @@ package org.apache.cassandra.net.async;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 /**
  * Identifies an outbound messaging connection.
  *
@@ -38,24 +40,24 @@ public class OutboundConnectionIdentifier
     /**
      * Memoization of the local node's broadcast address.
      */
-    private final InetSocketAddress localAddr;
+    private final InetAddressAndPort 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;
+    private final InetAddressAndPort 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 InetAddressAndPort remoteConnectionAddr;
 
     private final ConnectionType connectionType;
 
-    private OutboundConnectionIdentifier(InetSocketAddress localAddr,
-                                         InetSocketAddress remoteAddr,
-                                         InetSocketAddress remoteConnectionAddr,
+    private OutboundConnectionIdentifier(InetAddressAndPort localAddr,
+                                         InetAddressAndPort remoteAddr,
+                                         InetAddressAndPort remoteConnectionAddr,
                                          ConnectionType connectionType)
     {
         this.localAddr = localAddr;
@@ -64,8 +66,8 @@ public class OutboundConnectionIdentifier
         this.connectionType = connectionType;
     }
 
-    private OutboundConnectionIdentifier(InetSocketAddress localAddr,
-                                         InetSocketAddress remoteAddr,
+    private OutboundConnectionIdentifier(InetAddressAndPort localAddr,
+                                         InetAddressAndPort remoteAddr,
                                          ConnectionType connectionType)
     {
         this(localAddr, remoteAddr, remoteAddr, connectionType);
@@ -75,7 +77,7 @@ public class OutboundConnectionIdentifier
      * 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)
+    public static OutboundConnectionIdentifier small(InetAddressAndPort localAddr, InetAddressAndPort remoteAddr)
     {
         return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.SMALL_MESSAGE);
     }
@@ -84,7 +86,7 @@ public class OutboundConnectionIdentifier
      * 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)
+    public static OutboundConnectionIdentifier large(InetAddressAndPort localAddr, InetAddressAndPort remoteAddr)
     {
         return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.LARGE_MESSAGE);
     }
@@ -93,7 +95,7 @@ public class OutboundConnectionIdentifier
      * 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)
+    public static OutboundConnectionIdentifier gossip(InetAddressAndPort localAddr, InetAddressAndPort remoteAddr)
     {
         return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.GOSSIP);
     }
@@ -102,7 +104,7 @@ 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)
+    public static OutboundConnectionIdentifier stream(InetAddressAndPort localAddr, InetAddressAndPort remoteAddr)
     {
         return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.STREAM);
     }
@@ -115,45 +117,37 @@ 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.
      */
-    public OutboundConnectionIdentifier withNewConnectionAddress(InetSocketAddress remoteConnectionAddr)
+    public OutboundConnectionIdentifier withNewConnectionAddress(InetAddressAndPort remoteConnectionAddr)
     {
         return new OutboundConnectionIdentifier(localAddr, remoteAddr, remoteConnectionAddr, connectionType);
     }
 
     public OutboundConnectionIdentifier withNewConnectionPort(int port)
     {
-        return new OutboundConnectionIdentifier(localAddr, new InetSocketAddress(remoteAddr.getAddress(), port),
-                                                new InetSocketAddress(remoteConnectionAddr.getAddress(), port), connectionType);
+        return new OutboundConnectionIdentifier(localAddr, InetAddressAndPort.getByAddressOverrideDefaults(remoteAddr.address, port),
+                                                InetAddressAndPort.getByAddressOverrideDefaults(remoteConnectionAddr.address, port), connectionType);
     }
 
     /**
      * The local node address.
      */
-    public InetAddress local()
+    public InetAddressAndPort local()
     {
-        return localAddr.getAddress();
+        return localAddr;
     }
 
     /**
      * The remote node identifying address (the one to use for anything else than connecting to the node).
      */
-    public InetSocketAddress remoteAddress()
+    public  InetAddressAndPort remote()
     {
         return remoteAddr;
     }
 
     /**
-     * The remote node identifying address (the one to use for anything else than connecting to the node).
-     */
-    public  InetAddress remote()
-    {
-        return remoteAddr.getAddress();
-    }
-
-    /**
      * The remote node connection address (the one to use to actually connect to the remote, and only that).
      */
-    public InetSocketAddress connectionAddress()
+    public InetAddressAndPort connectionAddress()
     {
         return remoteConnectionAddr;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index 4522ba4..28775ef 100644
--- a/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
+++ b/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
@@ -43,6 +43,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.NettyFactory.Mode;
@@ -258,12 +259,12 @@ public class OutboundMessagingConnection
         logger.debug("connection attempt {} to {}", connectAttemptCount, connectionId);
 
 
-        InetSocketAddress remote = connectionId.remoteAddress();
-        if (!authenticator.authenticate(remote.getAddress(), remote.getPort()))
+        InetAddressAndPort remote = connectionId.remote();
+        if (!authenticator.authenticate(remote.address, remote.port))
         {
             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());
+            MessagingService.instance().destroyConnectionPool(remote);
 
             // don't update the state field as destroyConnectionPool() *should* call OMC.close()
             // on all the connections in the OMP for the remoteAddress
@@ -284,7 +285,7 @@ public class OutboundMessagingConnection
     }
 
     @VisibleForTesting
-    static boolean shouldCompressConnection(InetAddress localHost, InetAddress remoteHost)
+    static boolean shouldCompressConnection(InetAddressAndPort localHost, InetAddressAndPort remoteHost)
     {
         return (DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.all)
                || ((DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.dc) && !isLocalDC(localHost, remoteHost));
@@ -355,7 +356,7 @@ public class OutboundMessagingConnection
         return null;
     }
 
-    static boolean isLocalDC(InetAddress localHost, InetAddress remoteHost)
+    static boolean isLocalDC(InetAddressAndPort localHost, InetAddressAndPort remoteHost)
     {
         String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(remoteHost);
         String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(localHost);
@@ -585,7 +586,7 @@ public class OutboundMessagingConnection
      * 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)
+    void reconnectWithNewIp(InetAddressAndPort newAddr)
     {
         State currentState = state.get();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index 0086da8..c701229 100644
--- a/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java
+++ b/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java
@@ -28,6 +28,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.ConnectionMetrics;
 import org.apache.cassandra.net.BackPressureState;
 import org.apache.cassandra.net.MessageOut;
@@ -56,14 +57,14 @@ public class OutboundMessagingPool
      * 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;
+    private InetAddressAndPort preferredRemoteAddr;
 
-    public OutboundMessagingPool(InetSocketAddress remoteAddr, InetSocketAddress localAddr, ServerEncryptionOptions encryptionOptions,
+    public OutboundMessagingPool(InetAddressAndPort remoteAddr, InetAddressAndPort localAddr, ServerEncryptionOptions encryptionOptions,
                                  BackPressureState backPressureState, IInternodeAuthenticator authenticator)
     {
         preferredRemoteAddr = remoteAddr;
         this.backPressureState = backPressureState;
-        metrics = new ConnectionMetrics(localAddr.getAddress(), this);
+        metrics = new ConnectionMetrics(localAddr, this);
 
 
         smallMessageChannel = new OutboundMessagingConnection(OutboundConnectionIdentifier.small(localAddr, preferredRemoteAddr),
@@ -76,10 +77,10 @@ public class OutboundMessagingPool
                                                         encryptionOptions, Optional.empty(), authenticator);
     }
 
-    private static Optional<CoalescingStrategy> coalescingStrategy(InetSocketAddress remoteAddr)
+    private static Optional<CoalescingStrategy> coalescingStrategy(InetAddressAndPort remoteAddr)
     {
         String strategyName = DatabaseDescriptor.getOtcCoalescingStrategy();
-        String displayName = remoteAddr.getAddress().getHostAddress();
+        String displayName = remoteAddr.toString();
         return CoalescingStrategies.newCoalescingStrategy(strategyName,
                                                           DatabaseDescriptor.getOtcCoalescingWindow(),
                                                           OutboundMessagingConnection.logger,
@@ -117,7 +118,7 @@ public class OutboundMessagingPool
      *
      * @param addr IP Address to use (and prefer) going forward for connecting to the peer
      */
-    public void reconnectWithNewIp(InetSocketAddress addr)
+    public void reconnectWithNewIp(InetAddressAndPort addr)
     {
         preferredRemoteAddr = addr;
         gossipChannel.reconnectWithNewIp(addr);
@@ -166,7 +167,7 @@ public class OutboundMessagingPool
         return metrics.timeouts.getCount();
     }
 
-    public InetSocketAddress getPreferredRemoteAddr()
+    public InetAddressAndPort getPreferredRemoteAddr()
     {
         return preferredRemoteAddr;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java
index 5464520..2c4fae4 100644
--- a/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java
@@ -18,13 +18,13 @@
 
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.List;
 import java.util.UUID;
 
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamEvent;
@@ -41,15 +41,15 @@ public class AsymmetricLocalSyncTask extends AsymmetricSyncTask implements Strea
     private final UUID pendingRepair;
     private final TraceState state = Tracing.instance.get();
 
-    public AsymmetricLocalSyncTask(RepairJobDesc desc, InetAddress fetchFrom, List<Range<Token>> rangesToFetch, UUID pendingRepair, PreviewKind previewKind)
+    public AsymmetricLocalSyncTask(RepairJobDesc desc, InetAddressAndPort fetchFrom, List<Range<Token>> rangesToFetch, UUID pendingRepair, PreviewKind previewKind)
     {
-        super(desc, FBUtilities.getBroadcastAddress(), fetchFrom, rangesToFetch, previewKind);
+        super(desc, FBUtilities.getBroadcastAddressAndPort(), fetchFrom, rangesToFetch, previewKind);
         this.pendingRepair = pendingRepair;
     }
 
     public void startSync(List<Range<Token>> rangesToFetch)
     {
-        InetAddress preferred = SystemKeyspace.getPreferredIP(fetchFrom);
+        InetAddressAndPort preferred = SystemKeyspace.getPreferredIP(fetchFrom);
         StreamPlan plan = new StreamPlan(StreamOperation.REPAIR,
                                          1, false,
                                          false,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
index d70975d..e24d854 100644
--- a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
@@ -18,12 +18,12 @@
 
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.List;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.RepairException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.AsymmetricSyncRequest;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -33,14 +33,14 @@ import org.apache.cassandra.utils.FBUtilities;
 
 public class AsymmetricRemoteSyncTask extends AsymmetricSyncTask implements CompletableRemoteSyncTask
 {
-    public AsymmetricRemoteSyncTask(RepairJobDesc desc, InetAddress fetchNode, InetAddress fetchFrom, List<Range<Token>> rangesToFetch, PreviewKind previewKind)
+    public AsymmetricRemoteSyncTask(RepairJobDesc desc, InetAddressAndPort fetchNode, InetAddressAndPort fetchFrom, List<Range<Token>> rangesToFetch, PreviewKind previewKind)
     {
         super(desc, fetchNode, fetchFrom, rangesToFetch, previewKind);
     }
 
     public void startSync(List<Range<Token>> rangesToFetch)
     {
-        InetAddress local = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
         AsymmetricSyncRequest request = new AsymmetricSyncRequest(desc, local, fetchingNode, fetchFrom, rangesToFetch, previewKind);
         String message = String.format("Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", request.ranges.size(), request.fetchingNode, request.fetchFrom);
         Tracing.traceRepair(message);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java
index fe00058..4d38e8a 100644
--- a/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
@@ -30,6 +29,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.tracing.Tracing;
 
@@ -37,15 +37,15 @@ public abstract class AsymmetricSyncTask extends AbstractFuture<SyncStat> implem
 {
     private static Logger logger = LoggerFactory.getLogger(AsymmetricSyncTask.class);
     protected final RepairJobDesc desc;
-    protected final InetAddress fetchFrom;
+    protected final InetAddressAndPort fetchFrom;
     protected final List<Range<Token>> rangesToFetch;
-    protected final InetAddress fetchingNode;
+    protected final InetAddressAndPort fetchingNode;
     protected final PreviewKind previewKind;
     private long startTime = Long.MIN_VALUE;
     protected volatile SyncStat stat;
 
 
-    public AsymmetricSyncTask(RepairJobDesc desc, InetAddress fetchingNode, InetAddress fetchFrom, List<Range<Token>> rangesToFetch, PreviewKind previewKind)
+    public AsymmetricSyncTask(RepairJobDesc desc, InetAddressAndPort fetchingNode, InetAddressAndPort fetchFrom, List<Range<Token>> rangesToFetch, PreviewKind previewKind)
     {
         this.desc = desc;
         this.fetchFrom = fetchFrom;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/LocalSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/LocalSyncTask.java b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
index 8545b22..60d571b 100644
--- a/src/java/org/apache/cassandra/repair/LocalSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.List;
 import java.util.UUID;
 
@@ -28,7 +27,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamEvent;
@@ -39,8 +38,6 @@ import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.MerkleTree;
-import org.apache.cassandra.utils.MerkleTrees;
 
 /**
  * LocalSyncTask performs streaming between local(coordinator) node and remote replica.
@@ -62,7 +59,7 @@ public class LocalSyncTask extends SyncTask implements StreamEventHandler
     }
 
     @VisibleForTesting
-    StreamPlan createStreamPlan(InetAddress dst, InetAddress preferred, List<Range<Token>> differences)
+    StreamPlan createStreamPlan(InetAddressAndPort dst, InetAddressAndPort preferred, List<Range<Token>> differences)
     {
         StreamPlan plan = new StreamPlan(StreamOperation.REPAIR, 1, false, false, pendingRepair, previewKind)
                           .listeners(this)
@@ -84,10 +81,10 @@ public class LocalSyncTask extends SyncTask implements StreamEventHandler
     @Override
     protected void startSync(List<Range<Token>> differences)
     {
-        InetAddress local = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
         // We can take anyone of the node as source or destination, however if one is localhost, we put at source to avoid a forwarding
-        InetAddress dst = r2.endpoint.equals(local) ? r1.endpoint : r2.endpoint;
-        InetAddress preferred = SystemKeyspace.getPreferredIP(dst);
+        InetAddressAndPort dst = r2.endpoint.equals(local) ? r1.endpoint : r2.endpoint;
+        InetAddressAndPort preferred = SystemKeyspace.getPreferredIP(dst);
 
         String message = String.format("Performing streaming repair of %d ranges with %s", differences.size(), dst);
         logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/NodePair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/NodePair.java b/src/java/org/apache/cassandra/repair/NodePair.java
index a73c61a..bfb237e 100644
--- a/src/java/org/apache/cassandra/repair/NodePair.java
+++ b/src/java/org/apache/cassandra/repair/NodePair.java
@@ -18,13 +18,13 @@
 package org.apache.cassandra.repair;
 
 import java.io.IOException;
-import java.net.InetAddress;
 
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 
 /**
@@ -36,10 +36,10 @@ public class NodePair
 {
     public static IVersionedSerializer<NodePair> serializer = new NodePairSerializer();
 
-    public final InetAddress endpoint1;
-    public final InetAddress endpoint2;
+    public final InetAddressAndPort endpoint1;
+    public final InetAddressAndPort endpoint2;
 
-    public NodePair(InetAddress endpoint1, InetAddress endpoint2)
+    public NodePair(InetAddressAndPort endpoint1, InetAddressAndPort endpoint2)
     {
         this.endpoint1 = endpoint1;
         this.endpoint2 = endpoint2;
@@ -56,6 +56,12 @@ public class NodePair
     }
 
     @Override
+    public String toString()
+    {
+        return endpoint1.toString() + " - " + endpoint2.toString();
+    }
+
+    @Override
     public int hashCode()
     {
         return Objects.hashCode(endpoint1, endpoint2);
@@ -65,20 +71,21 @@ public class NodePair
     {
         public void serialize(NodePair nodePair, DataOutputPlus out, int version) throws IOException
         {
-            CompactEndpointSerializationHelper.serialize(nodePair.endpoint1, out);
-            CompactEndpointSerializationHelper.serialize(nodePair.endpoint2, out);
+            CompactEndpointSerializationHelper.instance.serialize(nodePair.endpoint1, out, version);
+            CompactEndpointSerializationHelper.instance.serialize(nodePair.endpoint2, out, version);
         }
 
         public NodePair deserialize(DataInputPlus in, int version) throws IOException
         {
-            InetAddress ep1 = CompactEndpointSerializationHelper.deserialize(in);
-            InetAddress ep2 = CompactEndpointSerializationHelper.deserialize(in);
+            InetAddressAndPort ep1 = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+            InetAddressAndPort ep2 = CompactEndpointSerializationHelper.instance.deserialize(in, version);
             return new NodePair(ep1, ep2);
         }
 
         public long serializedSize(NodePair nodePair, int version)
         {
-            return 2 * CompactEndpointSerializationHelper.serializedSize(nodePair.endpoint1);
+            return CompactEndpointSerializationHelper.instance.serializedSize(nodePair.endpoint1, version)
+                 + CompactEndpointSerializationHelper.instance.serializedSize(nodePair.endpoint2, version);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RemoteSyncTask.java b/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
index 93feb72..0a47f73 100644
--- a/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.List;
 
 import org.slf4j.Logger;
@@ -26,6 +25,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.RepairException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.SyncRequest;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -51,7 +51,7 @@ public class RemoteSyncTask extends SyncTask implements CompletableRemoteSyncTas
     @Override
     protected void startSync(List<Range<Token>> differences)
     {
-        InetAddress local = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
         SyncRequest request = new SyncRequest(desc, local, r1.endpoint, r2.endpoint, differences, previewKind);
         String message = String.format("Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", request.ranges.size(), request.src, request.dst);
         logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 7b8eb92..48973d2 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.*;
 import java.util.stream.Collectors;
 
@@ -33,6 +32,7 @@ import org.apache.cassandra.repair.asymmetric.DifferenceHolder;
 import org.apache.cassandra.repair.asymmetric.HostDifferences;
 import org.apache.cassandra.repair.asymmetric.PreferedNodeFilter;
 import org.apache.cassandra.repair.asymmetric.ReduceHelper;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
@@ -83,14 +83,14 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         Keyspace ks = Keyspace.open(desc.keyspace);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(desc.columnFamily);
         cfs.metric.repairsStarted.inc();
-        List<InetAddress> allEndpoints = new ArrayList<>(session.endpoints);
-        allEndpoints.add(FBUtilities.getBroadcastAddress());
+        List<InetAddressAndPort> allEndpoints = new ArrayList<>(session.endpoints);
+        allEndpoints.add(FBUtilities.getBroadcastAddressAndPort());
 
         ListenableFuture<List<TreeResponse>> validations;
         // Create a snapshot at all nodes unless we're using pure parallel repairs
         if (parallelismDegree != RepairParallelism.PARALLEL)
         {
-            ListenableFuture<List<InetAddress>> allSnapshotTasks;
+            ListenableFuture<List<InetAddressAndPort>> allSnapshotTasks;
             if (isIncremental)
             {
                 // consistent repair does it's own "snapshotting"
@@ -99,8 +99,8 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
             else
             {
                 // Request snapshot to all replica
-                List<ListenableFuture<InetAddress>> snapshotTasks = new ArrayList<>(allEndpoints.size());
-                for (InetAddress endpoint : allEndpoints)
+                List<ListenableFuture<InetAddressAndPort>> snapshotTasks = new ArrayList<>(allEndpoints.size());
+                for (InetAddressAndPort endpoint : allEndpoints)
                 {
                     SnapshotTask snapshotTask = new SnapshotTask(desc, endpoint);
                     snapshotTasks.add(snapshotTask);
@@ -110,9 +110,9 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
             }
 
             // When all snapshot complete, send validation requests
-            validations = Futures.transformAsync(allSnapshotTasks, new AsyncFunction<List<InetAddress>, List<TreeResponse>>()
+            validations = Futures.transformAsync(allSnapshotTasks, new AsyncFunction<List<InetAddressAndPort>, List<TreeResponse>>()
             {
-                public ListenableFuture<List<TreeResponse>> apply(List<InetAddress> endpoints)
+                public ListenableFuture<List<TreeResponse>> apply(List<InetAddressAndPort> endpoints)
                 {
                     if (parallelismDegree == RepairParallelism.SEQUENTIAL)
                         return sendSequentialValidationRequest(endpoints);
@@ -164,7 +164,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
     {
         return trees ->
         {
-            InetAddress local = FBUtilities.getLocalAddress();
+            InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
 
             List<SyncTask> syncTasks = new ArrayList<>();
             // We need to difference all trees one against another
@@ -198,7 +198,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
     {
         return trees ->
         {
-            InetAddress local = FBUtilities.getLocalAddress();
+            InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
 
             List<AsymmetricSyncTask> syncTasks = new ArrayList<>();
             // We need to difference all trees one against another
@@ -210,16 +210,16 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
                                                               .filter(node -> getDC(streaming)
                                                                               .equals(getDC(node)))
                                                               .collect(Collectors.toSet());
-            ImmutableMap<InetAddress, HostDifferences> reducedDifferences = ReduceHelper.reduce(diffHolder, preferSameDCFilter);
+            ImmutableMap<InetAddressAndPort, HostDifferences> reducedDifferences = ReduceHelper.reduce(diffHolder, preferSameDCFilter);
 
             for (int i = 0; i < trees.size(); i++)
             {
-                InetAddress address = trees.get(i).endpoint;
+                InetAddressAndPort address = trees.get(i).endpoint;
                 HostDifferences streamsFor = reducedDifferences.get(address);
                 if (streamsFor != null)
                 {
                     assert streamsFor.get(address).isEmpty() : "We should not fetch ranges from ourselves";
-                    for (InetAddress fetchFrom : streamsFor.hosts())
+                    for (InetAddressAndPort fetchFrom : streamsFor.hosts())
                     {
                         List<Range<Token>> toFetch = streamsFor.get(fetchFrom);
                         logger.debug("{} is about to fetch {} from {}", address, toFetch, fetchFrom);
@@ -246,7 +246,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         };
     }
 
-    private String getDC(InetAddress address)
+    private String getDC(InetAddressAndPort address)
     {
         return DatabaseDescriptor.getEndpointSnitch().getDatacenter(address);
     }
@@ -257,14 +257,14 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
      * @param endpoints Endpoint addresses to send validation request
      * @return Future that can get all {@link TreeResponse} from replica, if all validation succeed.
      */
-    private ListenableFuture<List<TreeResponse>> sendValidationRequest(Collection<InetAddress> endpoints)
+    private ListenableFuture<List<TreeResponse>> sendValidationRequest(Collection<InetAddressAndPort> endpoints)
     {
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
         Tracing.traceRepair(message);
         int nowInSec = FBUtilities.nowInSeconds();
         List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
-        for (InetAddress endpoint : endpoints)
+        for (InetAddressAndPort endpoint : endpoints)
         {
             ValidationTask task = new ValidationTask(desc, endpoint, nowInSec, previewKind);
             tasks.add(task);
@@ -277,7 +277,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
     /**
      * Creates {@link ValidationTask} and submit them to task executor so that tasks run sequentially.
      */
-    private ListenableFuture<List<TreeResponse>> sendSequentialValidationRequest(Collection<InetAddress> endpoints)
+    private ListenableFuture<List<TreeResponse>> sendSequentialValidationRequest(Collection<InetAddressAndPort> endpoints)
     {
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
@@ -285,8 +285,8 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         int nowInSec = FBUtilities.nowInSeconds();
         List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
 
-        Queue<InetAddress> requests = new LinkedList<>(endpoints);
-        InetAddress address = requests.poll();
+        Queue<InetAddressAndPort> requests = new LinkedList<>(endpoints);
+        InetAddressAndPort address = requests.poll();
         ValidationTask firstTask = new ValidationTask(desc, address, nowInSec, previewKind);
         logger.info("Validating {}", address);
         session.waitForValidation(Pair.create(desc, address), firstTask);
@@ -294,7 +294,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         ValidationTask currentTask = firstTask;
         while (requests.size() > 0)
         {
-            final InetAddress nextAddress = requests.poll();
+            final InetAddressAndPort nextAddress = requests.poll();
             final ValidationTask nextTask = new ValidationTask(desc, nextAddress, nowInSec, previewKind);
             tasks.add(nextTask);
             Futures.addCallback(currentTask, new FutureCallback<TreeResponse>()
@@ -319,7 +319,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
     /**
      * Creates {@link ValidationTask} and submit them to task executor so that tasks run sequentially within each dc.
      */
-    private ListenableFuture<List<TreeResponse>> sendDCAwareValidationRequest(Collection<InetAddress> endpoints)
+    private ListenableFuture<List<TreeResponse>> sendDCAwareValidationRequest(Collection<InetAddressAndPort> endpoints)
     {
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
@@ -327,11 +327,11 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         int nowInSec = FBUtilities.nowInSeconds();
         List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
 
-        Map<String, Queue<InetAddress>> requestsByDatacenter = new HashMap<>();
-        for (InetAddress endpoint : endpoints)
+        Map<String, Queue<InetAddressAndPort>> requestsByDatacenter = new HashMap<>();
+        for (InetAddressAndPort endpoint : endpoints)
         {
             String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint);
-            Queue<InetAddress> queue = requestsByDatacenter.get(dc);
+            Queue<InetAddressAndPort> queue = requestsByDatacenter.get(dc);
             if (queue == null)
             {
                 queue = new LinkedList<>();
@@ -340,10 +340,10 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
             queue.add(endpoint);
         }
 
-        for (Map.Entry<String, Queue<InetAddress>> entry : requestsByDatacenter.entrySet())
+        for (Map.Entry<String, Queue<InetAddressAndPort>> entry : requestsByDatacenter.entrySet())
         {
-            Queue<InetAddress> requests = entry.getValue();
-            InetAddress address = requests.poll();
+            Queue<InetAddressAndPort> requests = entry.getValue();
+            InetAddressAndPort address = requests.poll();
             ValidationTask firstTask = new ValidationTask(desc, address, nowInSec, previewKind);
             logger.info("Validating {}", address);
             session.waitForValidation(Pair.create(desc, address), firstTask);
@@ -351,7 +351,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
             ValidationTask currentTask = firstTask;
             while (requests.size() > 0)
             {
-                final InetAddress nextAddress = requests.poll();
+                final InetAddressAndPort nextAddress = requests.poll();
                 final ValidationTask nextTask = new ValidationTask(desc, nextAddress, nowInSec, previewKind);
                 tasks.add(nextTask);
                 Futures.addCallback(currentTask, new FutureCallback<TreeResponse>()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
index c26d4d1..4c0a564 100644
--- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
+++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.*;
 
 import com.google.common.base.Predicate;
@@ -28,10 +27,12 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.ParameterType;
 import org.apache.cassandra.repair.messages.*;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.ActiveRepairService;
@@ -225,11 +226,11 @@ public class RepairMessageVerbHandler implements IVerbHandler<RepairMessage>
         }
     }
 
-    private void logErrorAndSendFailureResponse(String errorMessage, InetAddress to, int id)
+    private void logErrorAndSendFailureResponse(String errorMessage, InetAddressAndPort to, int id)
     {
         logger.error(errorMessage);
         MessageOut reply = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
-                               .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE);
+                               .withParameter(ParameterType.FAILURE_RESPONSE, MessagingService.ONE_BYTE);
         MessagingService.instance().sendReply(reply, id, to);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index 1c9778b..5121874 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -60,6 +60,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.repair.consistent.CoordinatorSession;
 import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.QueryState;
@@ -141,10 +142,10 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
     @VisibleForTesting
     static class CommonRange
     {
-        public final Set<InetAddress> endpoints;
+        public final Set<InetAddressAndPort> endpoints;
         public final Collection<Range<Token>> ranges;
 
-        public CommonRange(Set<InetAddress> endpoints, Collection<Range<Token>> ranges)
+        public CommonRange(Set<InetAddressAndPort> endpoints, Collection<Range<Token>> ranges)
         {
             Preconditions.checkArgument(endpoints != null && !endpoints.isEmpty());
             Preconditions.checkArgument(ranges != null && !ranges.isEmpty());
@@ -232,7 +233,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
             traceState = null;
         }
 
-        final Set<InetAddress> allNeighbors = new HashSet<>();
+        final Set<InetAddressAndPort> allNeighbors = new HashSet<>();
         List<CommonRange> commonRanges = new ArrayList<>();
 
         //pre-calculate output of getLocalRanges and pass it to getNeighbors to increase performance and prevent
@@ -243,9 +244,9 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
         {
             for (Range<Token> range : options.getRanges())
             {
-                Set<InetAddress> neighbors = ActiveRepairService.getNeighbors(keyspace, keyspaceLocalRanges, range,
-                                                                              options.getDataCenters(),
-                                                                              options.getHosts());
+                Set<InetAddressAndPort> neighbors = ActiveRepairService.getNeighbors(keyspace, keyspaceLocalRanges, range,
+                                                                                     options.getDataCenters(),
+                                                                                     options.getHosts());
 
                 addRangeToNeighbors(commonRanges, range, neighbors);
                 allNeighbors.addAll(neighbors);
@@ -286,7 +287,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
 
         try (Timer.Context ctx = Keyspace.open(keyspace).metric.repairPrepareTime.time())
         {
-            ActiveRepairService.instance.prepareForRepair(parentSession, FBUtilities.getBroadcastAddress(), allNeighbors, options, columnFamilyStores);
+            ActiveRepairService.instance.prepareForRepair(parentSession, FBUtilities.getBroadcastAddressAndPort(), allNeighbors, options, columnFamilyStores);
             progress.incrementAndGet();
         }
         catch (Throwable t)
@@ -362,7 +363,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
      * removes dead nodes from common ranges, and exludes ranges left without any participants
      */
     @VisibleForTesting
-    static List<CommonRange> filterCommonRanges(List<CommonRange> commonRanges, Set<InetAddress> liveEndpoints, boolean force)
+    static List<CommonRange> filterCommonRanges(List<CommonRange> commonRanges, Set<InetAddressAndPort> liveEndpoints, boolean force)
     {
         if (!force)
         {
@@ -374,7 +375,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
 
             for (CommonRange commonRange: commonRanges)
             {
-                Set<InetAddress> endpoints = ImmutableSet.copyOf(Iterables.filter(commonRange.endpoints, liveEndpoints::contains));
+                Set<InetAddressAndPort> endpoints = ImmutableSet.copyOf(Iterables.filter(commonRange.endpoints, liveEndpoints::contains));
 
                 // this node is implicitly a participant in this repair, so a single endpoint is ok here
                 if (!endpoints.isEmpty())
@@ -391,15 +392,15 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
                                    long startTime,
                                    boolean forceRepair,
                                    TraceState traceState,
-                                   Set<InetAddress> allNeighbors,
+                                   Set<InetAddressAndPort> allNeighbors,
                                    List<CommonRange> commonRanges,
                                    String... cfnames)
     {
         // the local node also needs to be included in the set of participants, since coordinator sessions aren't persisted
-        Predicate<InetAddress> isAlive = FailureDetector.instance::isAlive;
-        Set<InetAddress> allParticipants = ImmutableSet.<InetAddress>builder()
+        Predicate<InetAddressAndPort> isAlive = FailureDetector.instance::isAlive;
+        Set<InetAddressAndPort> allParticipants = ImmutableSet.<InetAddressAndPort>builder()
                                            .addAll(forceRepair ? Iterables.filter(allNeighbors, isAlive) : allNeighbors)
-                                           .add(FBUtilities.getBroadcastAddress())
+                                           .add(FBUtilities.getBroadcastAddressAndPort())
                                            .build();
 
         List<CommonRange> allRanges = filterCommonRanges(commonRanges, allParticipants, forceRepair);
@@ -673,7 +674,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
                                                ImmutableList.of(failureMessage, completionMessage));
     }
 
-    private void addRangeToNeighbors(List<CommonRange> neighborRangeList, Range<Token> range, Set<InetAddress> neighbors)
+    private void addRangeToNeighbors(List<CommonRange> neighborRangeList, Range<Token> range, Set<InetAddressAndPort> neighbors)
     {
         for (int i = 0; i < neighborRangeList.size(); i++)
         {
@@ -708,7 +709,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
                 SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
 
                 ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
-                InetAddress source = FBUtilities.getBroadcastAddress();
+                InetAddressAndPort source = FBUtilities.getBroadcastAddressAndPort();
 
                 HashSet<UUID>[] seen = new HashSet[] { new HashSet<>(), new HashSet<>() };
                 int si = 0;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/RepairSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java
index 609ec56..91d767d 100644
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.repair;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -34,6 +33,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.*;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.SessionSummary;
 import org.apache.cassandra.tracing.Tracing;
@@ -95,14 +95,14 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
 
     /** Range to repair */
     public final Collection<Range<Token>> ranges;
-    public final Set<InetAddress> endpoints;
+    public final Set<InetAddressAndPort> endpoints;
     public final boolean isIncremental;
     public final PreviewKind previewKind;
 
     private final AtomicBoolean isFailed = new AtomicBoolean(false);
 
     // Each validation task waits response from replica in validating ConcurrentMap (keyed by CF name and endpoint address)
-    private final ConcurrentMap<Pair<RepairJobDesc, InetAddress>, ValidationTask> validating = new ConcurrentHashMap<>();
+    private final ConcurrentMap<Pair<RepairJobDesc, InetAddressAndPort>, ValidationTask> validating = new ConcurrentHashMap<>();
     // Remote syncing jobs wait response in syncingTasks map
     private final ConcurrentMap<Pair<RepairJobDesc, NodePair>, CompletableRemoteSyncTask> syncingTasks = new ConcurrentHashMap<>();
 
@@ -130,7 +130,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
                          Collection<Range<Token>> ranges,
                          String keyspace,
                          RepairParallelism parallelismDegree,
-                         Set<InetAddress> endpoints,
+                         Set<InetAddressAndPort> endpoints,
                          boolean isIncremental,
                          boolean pullRepair,
                          boolean force,
@@ -152,8 +152,8 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         if (force)
         {
             logger.debug("force flag set, removing dead endpoints");
-            final Set<InetAddress> removeCandidates = new HashSet<>();
-            for (final InetAddress endpoint : endpoints)
+            final Set<InetAddressAndPort> removeCandidates = new HashSet<>();
+            for (final InetAddressAndPort endpoint : endpoints)
             {
                 if (!FailureDetector.instance.isAlive(endpoint))
                 {
@@ -189,7 +189,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         return ranges;
     }
 
-    public void waitForValidation(Pair<RepairJobDesc, InetAddress> key, ValidationTask task)
+    public void waitForValidation(Pair<RepairJobDesc, InetAddressAndPort> key, ValidationTask task)
     {
         validating.put(key, task);
     }
@@ -207,7 +207,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
      * @param endpoint endpoint that sent merkle tree
      * @param trees calculated merkle trees, or null if validation failed
      */
-    public void validationComplete(RepairJobDesc desc, InetAddress endpoint, MerkleTrees trees)
+    public void validationComplete(RepairJobDesc desc, InetAddressAndPort endpoint, MerkleTrees trees)
     {
         ValidationTask task = validating.remove(Pair.create(desc, endpoint));
         if (task == null)
@@ -245,8 +245,8 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
     private String repairedNodes()
     {
         StringBuilder sb = new StringBuilder();
-        sb.append(FBUtilities.getBroadcastAddress());
-        for (InetAddress ep : endpoints)
+        sb.append(FBUtilities.getBroadcastAddressAndPort());
+        for (InetAddressAndPort ep : endpoints)
             sb.append(", ").append(ep);
         return sb.toString();
     }
@@ -285,7 +285,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         }
 
         // Checking all nodes are live
-        for (InetAddress endpoint : endpoints)
+        for (InetAddressAndPort endpoint : endpoints)
         {
             if (!FailureDetector.instance.isAlive(endpoint) && !skippedReplicas)
             {
@@ -353,23 +353,23 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         terminate();
     }
 
-    public void onJoin(InetAddress endpoint, EndpointState epState) {}
-    public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
-    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) {}
-    public void onAlive(InetAddress endpoint, EndpointState state) {}
-    public void onDead(InetAddress endpoint, EndpointState state) {}
+    public void onJoin(InetAddressAndPort endpoint, EndpointState epState) {}
+    public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
+    public void onChange(InetAddressAndPort endpoint, ApplicationState state, VersionedValue value) {}
+    public void onAlive(InetAddressAndPort endpoint, EndpointState state) {}
+    public void onDead(InetAddressAndPort endpoint, EndpointState state) {}
 
-    public void onRemove(InetAddress endpoint)
+    public void onRemove(InetAddressAndPort endpoint)
     {
         convict(endpoint, Double.MAX_VALUE);
     }
 
-    public void onRestart(InetAddress endpoint, EndpointState epState)
+    public void onRestart(InetAddressAndPort endpoint, EndpointState epState)
     {
         convict(endpoint, Double.MAX_VALUE);
     }
 
-    public void convict(InetAddress endpoint, double phi)
+    public void convict(InetAddressAndPort endpoint, double phi)
     {
         if (!endpoints.contains(endpoint))
             return;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/SnapshotTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SnapshotTask.java b/src/java/org/apache/cassandra/repair/SnapshotTask.java
index 2b267a7..acc5186 100644
--- a/src/java/org/apache/cassandra/repair/SnapshotTask.java
+++ b/src/java/org/apache/cassandra/repair/SnapshotTask.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.concurrent.RunnableFuture;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.util.concurrent.AbstractFuture;
 
 import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
@@ -32,12 +32,12 @@ import org.apache.cassandra.repair.messages.SnapshotMessage;
 /**
  * SnapshotTask is a task that sends snapshot request.
  */
-public class SnapshotTask extends AbstractFuture<InetAddress> implements RunnableFuture<InetAddress>
+public class SnapshotTask extends AbstractFuture<InetAddressAndPort> implements RunnableFuture<InetAddressAndPort>
 {
     private final RepairJobDesc desc;
-    private final InetAddress endpoint;
+    private final InetAddressAndPort endpoint;
 
-    public SnapshotTask(RepairJobDesc desc, InetAddress endpoint)
+    public SnapshotTask(RepairJobDesc desc, InetAddressAndPort endpoint)
     {
         this.desc = desc;
         this.endpoint = endpoint;
@@ -74,7 +74,7 @@ public class SnapshotTask extends AbstractFuture<InetAddress> implements Runnabl
 
         public boolean isLatencyForSnitch() { return false; }
 
-        public void onFailure(InetAddress from, RequestFailureReason failureReason)
+        public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
         {
             //listener.failedSnapshot();
             task.setException(new RuntimeException("Could not create snapshot at " + from));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
index a1b7459..59fee0b 100644
--- a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
+++ b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.UUID;
 import java.util.Collections;
 import java.util.Collection;
@@ -29,6 +28,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.SyncComplete;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -48,14 +49,14 @@ public class StreamingRepairTask implements Runnable, StreamEventHandler
 
     private final RepairJobDesc desc;
     private final boolean asymmetric;
-    private final InetAddress initiator;
-    private final InetAddress src;
-    private final InetAddress dst;
+    private final InetAddressAndPort initiator;
+    private final InetAddressAndPort src;
+    private final InetAddressAndPort dst;
     private final Collection<Range<Token>> ranges;
     private final UUID pendingRepair;
     private final PreviewKind previewKind;
 
-    public StreamingRepairTask(RepairJobDesc desc, InetAddress initiator, InetAddress src, InetAddress dst, Collection<Range<Token>> ranges,  UUID pendingRepair, PreviewKind previewKind, boolean asymmetric)
+    public StreamingRepairTask(RepairJobDesc desc, InetAddressAndPort initiator, InetAddressAndPort src, InetAddressAndPort dst, Collection<Range<Token>> ranges,  UUID pendingRepair, PreviewKind previewKind, boolean asymmetric)
     {
         this.desc = desc;
         this.initiator = initiator;
@@ -69,14 +70,14 @@ public class StreamingRepairTask implements Runnable, StreamEventHandler
 
     public void run()
     {
-        InetAddress dest = dst;
-        InetAddress preferred = SystemKeyspace.getPreferredIP(dest);
+        InetAddressAndPort dest = dst;
+        InetAddressAndPort preferred = SystemKeyspace.getPreferredIP(dest);
         logger.info("[streaming task #{}] Performing streaming repair of {} ranges with {}", desc.sessionId, ranges.size(), dst);
         createStreamPlan(dest, preferred).execute();
     }
 
     @VisibleForTesting
-    StreamPlan createStreamPlan(InetAddress dest, InetAddress preferred)
+    StreamPlan createStreamPlan(InetAddressAndPort dest, InetAddressAndPort preferred)
     {
         StreamPlan sp = new StreamPlan(StreamOperation.REPAIR, 1, false, false, pendingRepair, previewKind)
                .listeners(this)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
index 3770621..b46ae5e 100644
--- a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
+++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.repair;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -45,6 +44,7 @@ import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -74,48 +74,50 @@ public final class SystemDistributedKeyspace
 
     private static final TableMetadata RepairHistory =
         parse(REPAIR_HISTORY,
-              "Repair history",
-              "CREATE TABLE %s ("
-              + "keyspace_name text,"
-              + "columnfamily_name text,"
-              + "id timeuuid,"
-              + "parent_id timeuuid,"
-              + "range_begin text,"
-              + "range_end text,"
-              + "coordinator inet,"
-              + "participants set<inet>,"
-              + "exception_message text,"
-              + "exception_stacktrace text,"
-              + "status text,"
-              + "started_at timestamp,"
-              + "finished_at timestamp,"
-              + "PRIMARY KEY ((keyspace_name, columnfamily_name), id))");
+                "Repair history",
+                "CREATE TABLE %s ("
+                     + "keyspace_name text,"
+                     + "columnfamily_name text,"
+                     + "id timeuuid,"
+                     + "parent_id timeuuid,"
+                     + "range_begin text,"
+                     + "range_end text,"
+                     + "coordinator inet,"
+                     + "coordinator_port int,"
+                     + "participants set<inet>,"
+                     + "participants_v2 set<text>,"
+                     + "exception_message text,"
+                     + "exception_stacktrace text,"
+                     + "status text,"
+                     + "started_at timestamp,"
+                     + "finished_at timestamp,"
+                     + "PRIMARY KEY ((keyspace_name, columnfamily_name), id))");
 
     private static final TableMetadata ParentRepairHistory =
         parse(PARENT_REPAIR_HISTORY,
-              "Repair history",
-              "CREATE TABLE %s ("
-              + "parent_id timeuuid,"
-              + "keyspace_name text,"
-              + "columnfamily_names set<text>,"
-              + "started_at timestamp,"
-              + "finished_at timestamp,"
-              + "exception_message text,"
-              + "exception_stacktrace text,"
-              + "requested_ranges set<text>,"
-              + "successful_ranges set<text>,"
-              + "options map<text, text>,"
-              + "PRIMARY KEY (parent_id))");
+                "Repair history",
+                "CREATE TABLE %s ("
+                     + "parent_id timeuuid,"
+                     + "keyspace_name text,"
+                     + "columnfamily_names set<text>,"
+                     + "started_at timestamp,"
+                     + "finished_at timestamp,"
+                     + "exception_message text,"
+                     + "exception_stacktrace text,"
+                     + "requested_ranges set<text>,"
+                     + "successful_ranges set<text>,"
+                     + "options map<text, text>,"
+                     + "PRIMARY KEY (parent_id))");
 
     private static final TableMetadata ViewBuildStatus =
         parse(VIEW_BUILD_STATUS,
-              "Materialized View build status",
-              "CREATE TABLE %s ("
-              + "keyspace_name text,"
-              + "view_name text,"
-              + "host_id uuid,"
-              + "status text,"
-              + "PRIMARY KEY ((keyspace_name, view_name), host_id))");
+            "Materialized View build status",
+            "CREATE TABLE %s ("
+                     + "keyspace_name text,"
+                     + "view_name text,"
+                     + "host_id uuid,"
+                     + "status text,"
+                     + "PRIMARY KEY ((keyspace_name, view_name), host_id))");
 
     private static TableMetadata parse(String table, String description, String cql)
     {
@@ -184,17 +186,21 @@ public final class SystemDistributedKeyspace
         processSilent(fmtQuery);
     }
 
-    public static void startRepairs(UUID id, UUID parent_id, String keyspaceName, String[] cfnames, Collection<Range<Token>> ranges, Iterable<InetAddress> endpoints)
+    public static void startRepairs(UUID id, UUID parent_id, String keyspaceName, String[] cfnames, Collection<Range<Token>> ranges, Iterable<InetAddressAndPort> endpoints)
     {
-        String coordinator = FBUtilities.getBroadcastAddress().getHostAddress();
-        Set<String> participants = Sets.newHashSet(coordinator);
+        InetAddressAndPort coordinator = FBUtilities.getBroadcastAddressAndPort();
+        Set<String> participants = Sets.newHashSet();
+        Set<String> participants_v2 = Sets.newHashSet();
 
-        for (InetAddress endpoint : endpoints)
-            participants.add(endpoint.getHostAddress());
+        for (InetAddressAndPort endpoint : endpoints)
+        {
+            participants.add(endpoint.getHostAddress(false));
+            participants_v2.add(endpoint.toString());
+        }
 
         String query =
-                "INSERT INTO %s.%s (keyspace_name, columnfamily_name, id, parent_id, range_begin, range_end, coordinator, participants, status, started_at) " +
-                        "VALUES (   '%s',          '%s',              %s, %s,        '%s',        '%s',      '%s',        { '%s' },     '%s',   toTimestamp(now()))";
+                "INSERT INTO %s.%s (keyspace_name, columnfamily_name, id, parent_id, range_begin, range_end, coordinator, coordinator_port, participants, participants_v2, status, started_at) " +
+                        "VALUES (   '%s',          '%s',              %s, %s,        '%s',        '%s',      '%s',        %d,               { '%s' },     { '%s' },        '%s',   toTimestamp(now()))";
 
         for (String cfname : cfnames)
         {
@@ -207,8 +213,10 @@ public final class SystemDistributedKeyspace
                                               parent_id.toString(),
                                               range.left.toString(),
                                               range.right.toString(),
-                                              coordinator,
+                                              coordinator.getHostAddress(false),
+                                              coordinator.port,
                                               Joiner.on("', '").join(participants),
+                                              Joiner.on("', '").join(participants_v2),
                                               RepairState.STARTED.toString());
                 processSilent(fmtQry);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/TreeResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/TreeResponse.java b/src/java/org/apache/cassandra/repair/TreeResponse.java
index c898b36..8571caa 100644
--- a/src/java/org/apache/cassandra/repair/TreeResponse.java
+++ b/src/java/org/apache/cassandra/repair/TreeResponse.java
@@ -17,8 +17,7 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
-
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.MerkleTrees;
 
 /**
@@ -26,10 +25,10 @@ import org.apache.cassandra.utils.MerkleTrees;
  */
 public class TreeResponse
 {
-    public final InetAddress endpoint;
+    public final InetAddressAndPort endpoint;
     public final MerkleTrees trees;
 
-    public TreeResponse(InetAddress endpoint, MerkleTrees trees)
+    public TreeResponse(InetAddressAndPort endpoint, MerkleTrees trees)
     {
         this.endpoint = endpoint;
         this.trees = trees;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/ValidationTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/ValidationTask.java b/src/java/org/apache/cassandra/repair/ValidationTask.java
index 175709f..fc500cf 100644
--- a/src/java/org/apache/cassandra/repair/ValidationTask.java
+++ b/src/java/org/apache/cassandra/repair/ValidationTask.java
@@ -17,11 +17,10 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
-
 import com.google.common.util.concurrent.AbstractFuture;
 
 import org.apache.cassandra.exceptions.RepairException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.ValidationRequest;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -34,11 +33,11 @@ import org.apache.cassandra.utils.MerkleTrees;
 public class ValidationTask extends AbstractFuture<TreeResponse> implements Runnable
 {
     private final RepairJobDesc desc;
-    private final InetAddress endpoint;
+    private final InetAddressAndPort endpoint;
     private final int nowInSec;
     private final PreviewKind previewKind;
 
-    public ValidationTask(RepairJobDesc desc, InetAddress endpoint, int nowInSec, PreviewKind previewKind)
+    public ValidationTask(RepairJobDesc desc, InetAddressAndPort endpoint, int nowInSec, PreviewKind previewKind)
     {
         this.desc = desc;
         this.endpoint = endpoint;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/Validator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/Validator.java b/src/java/org/apache/cassandra/repair/Validator.java
index 9803638..4c2856d 100644
--- a/src/java/org/apache/cassandra/repair/Validator.java
+++ b/src/java/org/apache/cassandra/repair/Validator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
@@ -41,6 +40,7 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.ValidationComplete;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -64,7 +64,7 @@ public class Validator implements Runnable
     private static final Logger logger = LoggerFactory.getLogger(Validator.class);
 
     public final RepairJobDesc desc;
-    public final InetAddress initiator;
+    public final InetAddressAndPort initiator;
     public final int nowInSec;
     private final boolean evenTreeDistribution;
     public final boolean isIncremental;
@@ -81,17 +81,17 @@ public class Validator implements Runnable
 
     private final PreviewKind previewKind;
 
-    public Validator(RepairJobDesc desc, InetAddress initiator, int nowInSec, PreviewKind previewKind)
+    public Validator(RepairJobDesc desc, InetAddressAndPort initiator, int nowInSec, PreviewKind previewKind)
     {
         this(desc, initiator, nowInSec, false, false, previewKind);
     }
 
-    public Validator(RepairJobDesc desc, InetAddress initiator, int nowInSec, boolean isIncremental, PreviewKind previewKind)
+    public Validator(RepairJobDesc desc, InetAddressAndPort initiator, int nowInSec, boolean isIncremental, PreviewKind previewKind)
     {
         this(desc, initiator, nowInSec, false, isIncremental, previewKind);
     }
 
-    public Validator(RepairJobDesc desc, InetAddress initiator, int nowInSec, boolean evenTreeDistribution, boolean isIncremental, PreviewKind previewKind)
+    public Validator(RepairJobDesc desc, InetAddressAndPort initiator, int nowInSec, boolean evenTreeDistribution, boolean isIncremental, PreviewKind previewKind)
     {
         this.desc = desc;
         this.initiator = initiator;
@@ -352,7 +352,7 @@ public class Validator implements Runnable
     public void run()
     {
         // respond to the request that triggered this validation
-        if (!initiator.equals(FBUtilities.getBroadcastAddress()))
+        if (!initiator.equals(FBUtilities.getBroadcastAddressAndPort()))
         {
             logger.info("{} Sending completed merkle tree to {} for {}.{}", previewKind.logPrefix(desc.sessionId), initiator, desc.keyspace, desc.columnFamily);
             Tracing.traceRepair("Sending completed merkle tree to {} for {}.{}", initiator, desc.keyspace, desc.columnFamily);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java b/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java
index eb99977..c9b7ed7 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.asymmetric;
 
-import java.net.InetAddress;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -28,6 +27,7 @@ import com.google.common.collect.ImmutableMap;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.TreeResponse;
 import org.apache.cassandra.utils.MerkleTrees;
 
@@ -36,11 +36,11 @@ import org.apache.cassandra.utils.MerkleTrees;
  */
 public class DifferenceHolder
 {
-    private final ImmutableMap<InetAddress, HostDifferences> differences;
+    private final ImmutableMap<InetAddressAndPort, HostDifferences> differences;
 
     public DifferenceHolder(List<TreeResponse> trees)
     {
-        ImmutableMap.Builder<InetAddress, HostDifferences> diffBuilder = ImmutableMap.builder();
+        ImmutableMap.Builder<InetAddressAndPort, HostDifferences> diffBuilder = ImmutableMap.builder();
         for (int i = 0; i < trees.size() - 1; ++i)
         {
             TreeResponse r1 = trees.get(i);
@@ -58,9 +58,9 @@ public class DifferenceHolder
     }
 
     @VisibleForTesting
-    DifferenceHolder(Map<InetAddress, HostDifferences> differences)
+    DifferenceHolder(Map<InetAddressAndPort, HostDifferences> differences)
     {
-        ImmutableMap.Builder<InetAddress, HostDifferences> diffBuilder = ImmutableMap.builder();
+        ImmutableMap.Builder<InetAddressAndPort, HostDifferences> diffBuilder = ImmutableMap.builder();
         diffBuilder.putAll(differences);
         this.differences = diffBuilder.build();
     }
@@ -68,12 +68,12 @@ public class DifferenceHolder
     /**
      * differences only holds one 'side' of the difference - if A and B mismatch, only A will be a key in the map
      */
-    public Set<InetAddress> keyHosts()
+    public Set<InetAddressAndPort> keyHosts()
     {
         return differences.keySet();
     }
 
-    public HostDifferences get(InetAddress hostWithDifference)
+    public HostDifferences get(InetAddressAndPort hostWithDifference)
     {
         return differences.get(hostWithDifference);
     }
@@ -85,7 +85,7 @@ public class DifferenceHolder
                '}';
     }
 
-    public boolean hasDifferenceBetween(InetAddress node1, InetAddress node2, Range<Token> range)
+    public boolean hasDifferenceBetween(InetAddressAndPort node1, InetAddressAndPort node2, Range<Token> range)
     {
         HostDifferences diffsNode1 = differences.get(node1);
         if (diffsNode1 != null && diffsNode1.hasDifferencesFor(node2, range))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java b/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java
index 6cbe987..ab294b9 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.asymmetric;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -28,23 +27,24 @@ import java.util.Set;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Tracks the differences for a single host
  */
 public class HostDifferences
 {
-    private final Map<InetAddress, List<Range<Token>>> perHostDifferences = new HashMap<>();
+    private final Map<InetAddressAndPort, List<Range<Token>>> perHostDifferences = new HashMap<>();
 
     /**
      * Adds a set of differences between the node this instance is tracking and endpoint
      */
-    public void add(InetAddress endpoint, List<Range<Token>> difference)
+    public void add(InetAddressAndPort endpoint, List<Range<Token>> difference)
     {
         perHostDifferences.put(endpoint, difference);
     }
 
-    public void addSingleRange(InetAddress remoteNode, Range<Token> rangeToFetch)
+    public void addSingleRange(InetAddressAndPort remoteNode, Range<Token> rangeToFetch)
     {
         perHostDifferences.computeIfAbsent(remoteNode, (x) -> new ArrayList<>()).add(rangeToFetch);
     }
@@ -52,7 +52,7 @@ public class HostDifferences
     /**
      * Does this instance have differences for range with node2?
      */
-    public boolean hasDifferencesFor(InetAddress node2, Range<Token> range)
+    public boolean hasDifferencesFor(InetAddressAndPort node2, Range<Token> range)
     {
         List<Range<Token>> differences = get(node2);
         for (Range<Token> diff : differences)
@@ -64,12 +64,12 @@ public class HostDifferences
         return false;
     }
 
-    public Set<InetAddress> hosts()
+    public Set<InetAddressAndPort> hosts()
     {
         return perHostDifferences.keySet();
     }
 
-    public List<Range<Token>> get(InetAddress differingHost)
+    public List<Range<Token>> get(InetAddressAndPort differingHost)
     {
         return perHostDifferences.getOrDefault(differingHost, Collections.emptyList());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/asymmetric/IncomingRepairStreamTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/IncomingRepairStreamTracker.java b/src/java/org/apache/cassandra/repair/asymmetric/IncomingRepairStreamTracker.java
index b41ddd8..450336f 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/IncomingRepairStreamTracker.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/IncomingRepairStreamTracker.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.asymmetric;
 
-import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
@@ -29,6 +28,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Tracks incoming streams for a single host
@@ -60,7 +60,7 @@ public class IncomingRepairStreamTracker
      * @param range the range we need to stream from streamFromNode
      * @param streamFromNode the node we should stream from
      */
-    public void addIncomingRangeFrom(Range<Token> range, InetAddress streamFromNode)
+    public void addIncomingRangeFrom(Range<Token> range, InetAddressAndPort streamFromNode)
     {
         logger.trace("adding incoming range {} from {}", range, streamFromNode);
         Set<Range<Token>> newInput = RangeDenormalizer.denormalize(range, incoming);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java b/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java
index 90788dc..e8ca85d 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java
@@ -18,10 +18,11 @@
 
 package org.apache.cassandra.repair.asymmetric;
 
-import java.net.InetAddress;
 import java.util.Set;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 public interface PreferedNodeFilter
 {
-    public Set<InetAddress> apply(InetAddress streamingNode, Set<InetAddress> toStream);
+    public Set<InetAddressAndPort> apply(InetAddressAndPort streamingNode, Set<InetAddressAndPort> toStream);
 }
\ No newline at end of file


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


[18/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 4469384..91a1bff 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -35,6 +35,7 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.SetMultimap;
 import com.google.common.collect.Sets;
 import com.google.common.io.ByteStreams;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,6 +53,7 @@ import org.apache.cassandra.dht.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.*;
@@ -91,47 +93,52 @@ public final class SystemKeyspace
     public static final String PAXOS = "paxos";
     public static final String BUILT_INDEXES = "IndexInfo";
     public static final String LOCAL = "local";
-    public static final String PEERS = "peers";
-    public static final String PEER_EVENTS = "peer_events";
+    public static final String PEERS_V2 = "peers_v2";
+    public static final String PEER_EVENTS_V2 = "peer_events_v2";
     public static final String RANGE_XFERS = "range_xfers";
     public static final String COMPACTION_HISTORY = "compaction_history";
     public static final String SSTABLE_ACTIVITY = "sstable_activity";
     public static final String SIZE_ESTIMATES = "size_estimates";
     public static final String AVAILABLE_RANGES = "available_ranges";
     public static final String TRANSFERRED_RANGES = "transferred_ranges";
+    public static final String TRANSFERRED_RANGES_V2 = "transferred_ranges_v2";
     public static final String VIEW_BUILDS_IN_PROGRESS = "view_builds_in_progress";
     public static final String BUILT_VIEWS = "built_views";
     public static final String PREPARED_STATEMENTS = "prepared_statements";
     public static final String REPAIRS = "repairs";
 
+    @Deprecated public static final String LEGACY_PEERS = "peers";
+    @Deprecated public static final String LEGACY_PEER_EVENTS = "peer_events";
+    @Deprecated public static final String LEGACY_TRANSFERRED_RANGES = "transferred_ranges";
+
     public static final TableMetadata Batches =
         parse(BATCHES,
-              "batches awaiting replay",
-              "CREATE TABLE %s ("
-              + "id timeuuid,"
-              + "mutations list<blob>,"
-              + "version int,"
-              + "PRIMARY KEY ((id)))")
-              .partitioner(new LocalPartitioner(TimeUUIDType.instance))
-              .compaction(CompactionParams.scts(singletonMap("min_threshold", "2")))
-              .build();
+                "batches awaiting replay",
+                "CREATE TABLE %s ("
+                + "id timeuuid,"
+                + "mutations list<blob>,"
+                + "version int,"
+                + "PRIMARY KEY ((id)))")
+                .partitioner(new LocalPartitioner(TimeUUIDType.instance))
+                .compaction(CompactionParams.scts(singletonMap("min_threshold", "2")))
+                .build();
 
     private static final TableMetadata Paxos =
         parse(PAXOS,
-              "in-progress paxos proposals",
-              "CREATE TABLE %s ("
-              + "row_key blob,"
-              + "cf_id UUID,"
-              + "in_progress_ballot timeuuid,"
-              + "most_recent_commit blob,"
-              + "most_recent_commit_at timeuuid,"
-              + "most_recent_commit_version int,"
-              + "proposal blob,"
-              + "proposal_ballot timeuuid,"
-              + "proposal_version int,"
-              + "PRIMARY KEY ((row_key), cf_id))")
-              .compaction(CompactionParams.lcs(emptyMap()))
-              .build();
+                "in-progress paxos proposals",
+                "CREATE TABLE %s ("
+                + "row_key blob,"
+                + "cf_id UUID,"
+                + "in_progress_ballot timeuuid,"
+                + "most_recent_commit blob,"
+                + "most_recent_commit_at timeuuid,"
+                + "most_recent_commit_version int,"
+                + "proposal blob,"
+                + "proposal_ballot timeuuid,"
+                + "proposal_version int,"
+                + "PRIMARY KEY ((row_key), cf_id))")
+                .compaction(CompactionParams.lcs(emptyMap()))
+                .build();
 
     private static final TableMetadata BuiltIndexes =
         parse(BUILT_INDEXES,
@@ -145,122 +152,130 @@ public final class SystemKeyspace
 
     private static final TableMetadata Local =
         parse(LOCAL,
-              "information about the local node",
-              "CREATE TABLE %s ("
-              + "key text,"
-              + "bootstrapped text,"
-              + "broadcast_address inet,"
-              + "cluster_name text,"
-              + "cql_version text,"
-              + "data_center text,"
-              + "gossip_generation int,"
-              + "host_id uuid,"
-              + "listen_address inet,"
-              + "native_protocol_version text,"
-              + "partitioner text,"
-              + "rack text,"
-              + "release_version text,"
-              + "rpc_address inet,"
-              + "schema_version uuid,"
-              + "tokens set<varchar>,"
-              + "truncated_at map<uuid, blob>,"
-              + "PRIMARY KEY ((key)))")
-              .recordDeprecatedSystemColumn("thrift_version", UTF8Type.instance)
-              .build();
-
-    private static final TableMetadata Peers =
-        parse(PEERS,
-              "information about known peers in the cluster",
-              "CREATE TABLE %s ("
-              + "peer inet,"
-              + "data_center text,"
-              + "host_id uuid,"
-              + "preferred_ip inet,"
-              + "rack text,"
-              + "release_version text,"
-              + "rpc_address inet,"
-              + "schema_version uuid,"
-              + "tokens set<varchar>,"
-              + "PRIMARY KEY ((peer)))")
-              .build();
-
-    private static final TableMetadata PeerEvents =
-        parse(PEER_EVENTS,
-              "events related to peers",
-              "CREATE TABLE %s ("
-              + "peer inet,"
-              + "hints_dropped map<uuid, int>,"
-              + "PRIMARY KEY ((peer)))")
-              .build();
+                "information about the local node",
+                "CREATE TABLE %s ("
+                + "key text,"
+                + "bootstrapped text,"
+                + "broadcast_address inet,"
+                + "broadcast_port int,"
+                + "cluster_name text,"
+                + "cql_version text,"
+                + "data_center text,"
+                + "gossip_generation int,"
+                + "host_id uuid,"
+                + "listen_address inet,"
+                + "listen_port int,"
+                + "native_protocol_version text,"
+                + "partitioner text,"
+                + "rack text,"
+                + "release_version text,"
+                + "rpc_address inet,"
+                + "rpc_port int,"
+                + "schema_version uuid,"
+                + "tokens set<varchar>,"
+                + "truncated_at map<uuid, blob>,"
+                + "PRIMARY KEY ((key)))"
+                ).recordDeprecatedSystemColumn("thrift_version", UTF8Type.instance)
+                .build();
+
+    private static final TableMetadata PeersV2 =
+        parse(PEERS_V2,
+                "information about known peers in the cluster",
+                "CREATE TABLE %s ("
+                + "peer inet,"
+                + "peer_port int,"
+                + "data_center text,"
+                + "host_id uuid,"
+                + "preferred_ip inet,"
+                + "preferred_port int,"
+                + "rack text,"
+                + "release_version text,"
+                + "native_address inet,"
+                + "native_port int,"
+                + "schema_version uuid,"
+                + "tokens set<varchar>,"
+                + "PRIMARY KEY ((peer), peer_port))")
+                .build();
+
+    private static final TableMetadata PeerEventsV2 =
+        parse(PEER_EVENTS_V2,
+                "events related to peers",
+                "CREATE TABLE %s ("
+                + "peer inet,"
+                + "peer_port int,"
+                + "hints_dropped map<uuid, int>,"
+                + "PRIMARY KEY ((peer), peer_port))")
+                .build();
 
     private static final TableMetadata RangeXfers =
         parse(RANGE_XFERS,
-              "ranges requested for transfer",
-              "CREATE TABLE %s ("
-              + "token_bytes blob,"
-              + "requested_at timestamp,"
-              + "PRIMARY KEY ((token_bytes)))")
-              .build();
+                "ranges requested for transfer",
+                "CREATE TABLE %s ("
+                + "token_bytes blob,"
+                + "requested_at timestamp,"
+                + "PRIMARY KEY ((token_bytes)))")
+                .build();
 
     private static final TableMetadata CompactionHistory =
         parse(COMPACTION_HISTORY,
-              "week-long compaction history",
-              "CREATE TABLE %s ("
-              + "id uuid,"
-              + "bytes_in bigint,"
-              + "bytes_out bigint,"
-              + "columnfamily_name text,"
-              + "compacted_at timestamp,"
-              + "keyspace_name text,"
-              + "rows_merged map<int, bigint>,"
-              + "PRIMARY KEY ((id)))")
-              .defaultTimeToLive((int) TimeUnit.DAYS.toSeconds(7))
-              .build();
+                "week-long compaction history",
+                "CREATE TABLE %s ("
+                + "id uuid,"
+                + "bytes_in bigint,"
+                + "bytes_out bigint,"
+                + "columnfamily_name text,"
+                + "compacted_at timestamp,"
+                + "keyspace_name text,"
+                + "rows_merged map<int, bigint>,"
+                + "PRIMARY KEY ((id)))")
+                .defaultTimeToLive((int) TimeUnit.DAYS.toSeconds(7))
+                .build();
 
     private static final TableMetadata SSTableActivity =
         parse(SSTABLE_ACTIVITY,
-              "historic sstable read rates",
-              "CREATE TABLE %s ("
-              + "keyspace_name text,"
-              + "columnfamily_name text,"
-              + "generation int,"
-              + "rate_120m double,"
-              + "rate_15m double,"
-              + "PRIMARY KEY ((keyspace_name, columnfamily_name, generation)))")
-              .build();
+                "historic sstable read rates",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "columnfamily_name text,"
+                + "generation int,"
+                + "rate_120m double,"
+                + "rate_15m double,"
+                + "PRIMARY KEY ((keyspace_name, columnfamily_name, generation)))")
+                .build();
 
     private static final TableMetadata SizeEstimates =
         parse(SIZE_ESTIMATES,
-              "per-table primary range size estimates",
-              "CREATE TABLE %s ("
-              + "keyspace_name text,"
-              + "table_name text,"
-              + "range_start text,"
-              + "range_end text,"
-              + "mean_partition_size bigint,"
-              + "partitions_count bigint,"
-              + "PRIMARY KEY ((keyspace_name), table_name, range_start, range_end))")
-              .build();
+                "per-table primary range size estimates",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "table_name text,"
+                + "range_start text,"
+                + "range_end text,"
+                + "mean_partition_size bigint,"
+                + "partitions_count bigint,"
+                + "PRIMARY KEY ((keyspace_name), table_name, range_start, range_end))")
+                .build();
 
     private static final TableMetadata AvailableRanges =
         parse(AVAILABLE_RANGES,
-              "available keyspace/ranges during bootstrap/replace that are ready to be served",
-              "CREATE TABLE %s ("
-              + "keyspace_name text,"
-              + "ranges set<blob>,"
-              + "PRIMARY KEY ((keyspace_name)))")
-              .build();
-
-    private static final TableMetadata TransferredRanges =
-        parse(TRANSFERRED_RANGES,
-              "record of transferred ranges for streaming operation",
-              "CREATE TABLE %s ("
-              + "operation text,"
-              + "peer inet,"
-              + "keyspace_name text,"
-              + "ranges set<blob>,"
-              + "PRIMARY KEY ((operation, keyspace_name), peer))")
-              .build();
+                "available keyspace/ranges during bootstrap/replace that are ready to be served",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "ranges set<blob>,"
+                + "PRIMARY KEY ((keyspace_name)))")
+                .build();
+
+    private static final TableMetadata TransferredRangesV2 =
+        parse(TRANSFERRED_RANGES_V2,
+                "record of transferred ranges for streaming operation",
+                "CREATE TABLE %s ("
+                + "operation text,"
+                + "peer inet,"
+                + "peer_port int,"
+                + "keyspace_name text,"
+                + "ranges set<blob>,"
+                + "PRIMARY KEY ((operation, keyspace_name), peer, peer_port))")
+                .build();
 
     private static final TableMetadata ViewBuildsInProgress =
         parse(VIEW_BUILDS_IN_PROGRESS,
@@ -277,38 +292,79 @@ public final class SystemKeyspace
 
     private static final TableMetadata BuiltViews =
         parse(BUILT_VIEWS,
-              "built views",
-              "CREATE TABLE %s ("
-              + "keyspace_name text,"
-              + "view_name text,"
-              + "status_replicated boolean,"
-              + "PRIMARY KEY ((keyspace_name), view_name))")
-              .build();
+                "built views",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "view_name text,"
+                + "status_replicated boolean,"
+                + "PRIMARY KEY ((keyspace_name), view_name))")
+                .build();
 
     private static final TableMetadata PreparedStatements =
         parse(PREPARED_STATEMENTS,
-              "prepared statements",
-              "CREATE TABLE %s ("
-              + "prepared_id blob,"
-              + "logged_keyspace text,"
-              + "query_string text,"
-              + "PRIMARY KEY ((prepared_id)))")
-              .build();
+                "prepared statements",
+                "CREATE TABLE %s ("
+                + "prepared_id blob,"
+                + "logged_keyspace text,"
+                + "query_string text,"
+                + "PRIMARY KEY ((prepared_id)))")
+                .build();
 
     private static final TableMetadata Repairs =
         parse(REPAIRS,
-              "repairs",
-              "CREATE TABLE %s ("
-              + "parent_id timeuuid, "
-              + "started_at timestamp, "
-              + "last_update timestamp, "
-              + "repaired_at timestamp, "
-              + "state int, "
-              + "coordinator inet, "
-              + "participants set<inet>, "
-              + "ranges set<blob>, "
-              + "cfids set<uuid>, "
-              + "PRIMARY KEY (parent_id))").build();
+          "repairs",
+          "CREATE TABLE %s ("
+          + "parent_id timeuuid, "
+          + "started_at timestamp, "
+          + "last_update timestamp, "
+          + "repaired_at timestamp, "
+          + "state int, "
+          + "coordinator inet, "
+          + "coordinator_port int,"
+          + "participants set<inet>,"
+          + "participants_wp set<text>,"
+          + "ranges set<blob>, "
+          + "cfids set<uuid>, "
+          + "PRIMARY KEY (parent_id))").build();
+
+    @Deprecated
+    private static final TableMetadata LegacyPeers =
+        parse(LEGACY_PEERS,
+            "information about known peers in the cluster",
+            "CREATE TABLE %s ("
+            + "peer inet,"
+            + "data_center text,"
+            + "host_id uuid,"
+            + "preferred_ip inet,"
+            + "rack text,"
+            + "release_version text,"
+            + "rpc_address inet,"
+            + "schema_version uuid,"
+            + "tokens set<varchar>,"
+            + "PRIMARY KEY ((peer)))")
+            .build();
+
+    @Deprecated
+    private static final TableMetadata LegacyPeerEvents =
+        parse(LEGACY_PEER_EVENTS,
+            "events related to peers",
+            "CREATE TABLE %s ("
+            + "peer inet,"
+            + "hints_dropped map<uuid, int>,"
+            + "PRIMARY KEY ((peer)))")
+            .build();
+
+    @Deprecated
+    private static final TableMetadata LegacyTransferredRanges =
+        parse(LEGACY_TRANSFERRED_RANGES,
+            "record of transferred ranges for streaming operation",
+            "CREATE TABLE %s ("
+            + "operation text,"
+            + "peer inet,"
+            + "keyspace_name text,"
+            + "ranges set<blob>,"
+            + "PRIMARY KEY ((operation, keyspace_name), peer))")
+            .build();
 
     private static TableMetadata.Builder parse(String table, String description, String cql)
     {
@@ -331,14 +387,17 @@ public final class SystemKeyspace
                          Batches,
                          Paxos,
                          Local,
-                         Peers,
-                         PeerEvents,
+                         PeersV2,
+                         LegacyPeers,
+                         PeerEventsV2,
+                         LegacyPeerEvents,
                          RangeXfers,
                          CompactionHistory,
                          SSTableActivity,
                          SizeEstimates,
                          AvailableRanges,
-                         TransferredRanges,
+                         TransferredRangesV2,
+                         LegacyTransferredRanges,
                          ViewBuildsInProgress,
                          BuiltViews,
                          PreparedStatements,
@@ -384,9 +443,12 @@ public final class SystemKeyspace
                      "rack," +
                      "partitioner," +
                      "rpc_address," +
+                     "rpc_port," +
                      "broadcast_address," +
-                     "listen_address" +
-                     ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                     "broadcast_port," +
+                     "listen_address," +
+                     "listen_port" +
+                     ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
         executeOnceInternal(format(req, LOCAL),
                             LOCAL,
@@ -394,12 +456,15 @@ public final class SystemKeyspace
                             FBUtilities.getReleaseVersionString(),
                             QueryProcessor.CQL_VERSION.toString(),
                             String.valueOf(ProtocolVersion.CURRENT.asInt()),
-                            snitch.getDatacenter(FBUtilities.getBroadcastAddress()),
-                            snitch.getRack(FBUtilities.getBroadcastAddress()),
+                            snitch.getDatacenter(FBUtilities.getBroadcastAddressAndPort()),
+                            snitch.getRack(FBUtilities.getBroadcastAddressAndPort()),
                             DatabaseDescriptor.getPartitioner().getClass().getName(),
                             DatabaseDescriptor.getRpcAddress(),
-                            FBUtilities.getBroadcastAddress(),
-                            FBUtilities.getLocalAddress());
+                            DatabaseDescriptor.getNativeTransportPort(),
+                            FBUtilities.getJustBroadcastAddress(),
+                            DatabaseDescriptor.getStoragePort(),
+                            FBUtilities.getJustLocalAddress(),
+                            DatabaseDescriptor.getStoragePort());
     }
 
     public static void updateCompactionHistory(String ksname,
@@ -461,11 +526,10 @@ public final class SystemKeyspace
     {
         String buildReq = "DELETE FROM %s.%s WHERE keyspace_name = ? AND view_name = ?";
         executeInternal(String.format(buildReq, SchemaConstants.SYSTEM_KEYSPACE_NAME, VIEW_BUILDS_IN_PROGRESS), keyspaceName, viewName);
-        forceBlockingFlush(VIEW_BUILDS_IN_PROGRESS);
 
         String builtReq = "DELETE FROM %s.\"%s\" WHERE keyspace_name = ? AND view_name = ? IF EXISTS";
         executeInternal(String.format(builtReq, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_VIEWS), keyspaceName, viewName);
-        forceBlockingFlush(BUILT_VIEWS);
+        forceBlockingFlush(VIEW_BUILDS_IN_PROGRESS, BUILT_VIEWS);
     }
 
     public static void finishViewBuildStatus(String ksname, String viewName)
@@ -609,39 +673,64 @@ public final class SystemKeyspace
     /**
      * Record tokens being used by another node
      */
-    public static synchronized void updateTokens(InetAddress ep, Collection<Token> tokens)
+    public static synchronized void updateTokens(InetAddressAndPort ep, Collection<Token> tokens)
     {
-        if (ep.equals(FBUtilities.getBroadcastAddress()))
+        if (ep.equals(FBUtilities.getBroadcastAddressAndPort()))
             return;
 
         String req = "INSERT INTO system.%s (peer, tokens) VALUES (?, ?)";
-        executeInternal(format(req, PEERS), ep, tokensAsSet(tokens));
+        executeInternal(String.format(req, LEGACY_PEERS), ep.address, tokensAsSet(tokens));
+        req = "INSERT INTO system.%s (peer, peer_port, tokens) VALUES (?, ?, ?)";
+        executeInternal(String.format(req, PEERS_V2), ep.address, ep.port, tokensAsSet(tokens));
     }
 
-    public static synchronized void updatePreferredIP(InetAddress ep, InetAddress preferred_ip)
+    public static synchronized void updatePreferredIP(InetAddressAndPort ep, InetAddressAndPort preferred_ip)
     {
         if (getPreferredIP(ep) == preferred_ip)
             return;
 
         String req = "INSERT INTO system.%s (peer, preferred_ip) VALUES (?, ?)";
-        executeInternal(format(req, PEERS), ep, preferred_ip);
-        forceBlockingFlush(PEERS);
+        executeInternal(String.format(req, LEGACY_PEERS), ep.address, preferred_ip.address);
+        req = "INSERT INTO system.%s (peer, peer_port, preferred_ip, preferred_port) VALUES (?, ?, ?, ?)";
+        executeInternal(String.format(req, PEERS_V2), ep.address, ep.port, preferred_ip.address, preferred_ip.port);
+        forceBlockingFlush(LEGACY_PEERS, PEERS_V2);
     }
 
-    public static synchronized void updatePeerInfo(InetAddress ep, String columnName, Object value)
+    public static synchronized void updatePeerInfo(InetAddressAndPort ep, String columnName, Object value)
     {
-        if (ep.equals(FBUtilities.getBroadcastAddress()))
+        if (ep.equals(FBUtilities.getBroadcastAddressAndPort()))
             return;
 
         String req = "INSERT INTO system.%s (peer, %s) VALUES (?, ?)";
-        executeInternal(format(req, PEERS, columnName), ep, value);
+        executeInternal(String.format(req, LEGACY_PEERS, columnName), ep.address, value);
+        //This column doesn't match across the two tables
+        if (columnName.equals("rpc_address"))
+        {
+            columnName = "native_address";
+        }
+        req = "INSERT INTO system.%s (peer, peer_port, %s) VALUES (?, ?, ?)";
+        executeInternal(String.format(req, PEERS_V2, columnName), ep.address, ep.port, value);
     }
 
-    public static synchronized void updateHintsDropped(InetAddress ep, UUID timePeriod, int value)
+    public static synchronized void updatePeerNativeAddress(InetAddressAndPort ep, InetAddressAndPort address)
+    {
+        if (ep.equals(FBUtilities.getBroadcastAddressAndPort()))
+            return;
+
+        String req = "INSERT INTO system.%s (peer, rpc_address) VALUES (?, ?)";
+        executeInternal(String.format(req, LEGACY_PEERS), ep.address, address.address);
+        req = "INSERT INTO system.%s (peer, peer_port, native_address, native_port) VALUES (?, ?, ?, ?)";
+        executeInternal(String.format(req, PEERS_V2), ep.address, ep.port, address.address, address.port);
+    }
+
+
+    public static synchronized void updateHintsDropped(InetAddressAndPort ep, UUID timePeriod, int value)
     {
         // with 30 day TTL
         String req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ ? ] = ? WHERE peer = ?";
-        executeInternal(format(req, PEER_EVENTS), timePeriod, value, ep);
+        executeInternal(String.format(req, LEGACY_PEER_EVENTS), timePeriod, value, ep.address);
+        req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ ? ] = ? WHERE peer = ? AND peer_port = ?";
+        executeInternal(String.format(req, PEER_EVENTS_V2), timePeriod, value, ep.address, ep.port);
     }
 
     public static synchronized void updateSchemaVersion(UUID version)
@@ -673,11 +762,13 @@ public final class SystemKeyspace
     /**
      * Remove stored tokens being used by another node
      */
-    public static synchronized void removeEndpoint(InetAddress ep)
+    public static synchronized void removeEndpoint(InetAddressAndPort ep)
     {
         String req = "DELETE FROM system.%s WHERE peer = ?";
-        executeInternal(format(req, PEERS), ep);
-        forceBlockingFlush(PEERS);
+        executeInternal(String.format(req, LEGACY_PEERS), ep.address);
+        req = String.format("DELETE FROM system.%s WHERE peer = ? AND peer_port = ?", PEERS_V2);
+        executeInternal(req, ep.address, ep.port);
+        forceBlockingFlush(LEGACY_PEERS, PEERS_V2);
     }
 
     /**
@@ -696,22 +787,32 @@ public final class SystemKeyspace
         forceBlockingFlush(LOCAL);
     }
 
-    public static void forceBlockingFlush(String cfname)
+    public static void forceBlockingFlush(String ...cfnames)
     {
         if (!DatabaseDescriptor.isUnsafeSystem())
-            FBUtilities.waitOnFuture(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(cfname).forceFlush());
+        {
+            List<ListenableFuture<CommitLogPosition>> futures = new ArrayList<>();
+
+            for (String cfname : cfnames)
+            {
+                futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(cfname).forceFlush());
+            }
+            FBUtilities.waitOnFutures(futures);
+        }
     }
 
     /**
      * Return a map of stored tokens to IP addresses
      *
      */
-    public static SetMultimap<InetAddress, Token> loadTokens()
+    public static SetMultimap<InetAddressAndPort, Token> loadTokens()
     {
-        SetMultimap<InetAddress, Token> tokenMap = HashMultimap.create();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, tokens FROM system." + PEERS))
+        SetMultimap<InetAddressAndPort, Token> tokenMap = HashMultimap.create();
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, peer_port, tokens FROM system." + PEERS_V2))
         {
-            InetAddress peer = row.getInetAddress("peer");
+            InetAddress address = row.getInetAddress("peer");
+            Integer port = row.getInt("peer_port");
+            InetAddressAndPort peer = InetAddressAndPort.getByAddressOverrideDefaults(address, port);
             if (row.has("tokens"))
                 tokenMap.putAll(peer, deserializeTokens(row.getSet("tokens", UTF8Type.instance)));
         }
@@ -723,12 +824,14 @@ public final class SystemKeyspace
      * Return a map of store host_ids to IP addresses
      *
      */
-    public static Map<InetAddress, UUID> loadHostIds()
+    public static Map<InetAddressAndPort, UUID> loadHostIds()
     {
-        Map<InetAddress, UUID> hostIdMap = new HashMap<>();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, host_id FROM system." + PEERS))
+        Map<InetAddressAndPort, UUID> hostIdMap = new HashMap<>();
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, peer_port, host_id FROM system." + PEERS_V2))
         {
-            InetAddress peer = row.getInetAddress("peer");
+            InetAddress address = row.getInetAddress("peer");
+            Integer port = row.getInt("peer_port");
+            InetAddressAndPort peer = InetAddressAndPort.getByAddressOverrideDefaults(address, port);
             if (row.has("host_id"))
             {
                 hostIdMap.put(peer, row.getUUID("host_id"));
@@ -743,24 +846,29 @@ public final class SystemKeyspace
      * @param ep endpoint address to check
      * @return Preferred IP for given endpoint if present, otherwise returns given ep
      */
-    public static InetAddress getPreferredIP(InetAddress ep)
+    public static InetAddressAndPort getPreferredIP(InetAddressAndPort ep)
     {
-        String req = "SELECT preferred_ip FROM system.%s WHERE peer=?";
-        UntypedResultSet result = executeInternal(format(req, PEERS), ep);
+        String req = "SELECT preferred_ip, preferred_port FROM system.%s WHERE peer=? AND peer_port = ?";
+        UntypedResultSet result = executeInternal(String.format(req, PEERS_V2), ep.address, ep.port);
         if (!result.isEmpty() && result.one().has("preferred_ip"))
-            return result.one().getInetAddress("preferred_ip");
+        {
+            UntypedResultSet.Row row = result.one();
+            return InetAddressAndPort.getByAddressOverrideDefaults(row.getInetAddress("preferred_ip"), row.getInt("preferred_port"));
+        }
         return ep;
     }
 
     /**
      * Return a map of IP addresses containing a map of dc and rack info
      */
-    public static Map<InetAddress, Map<String,String>> loadDcRackInfo()
+    public static Map<InetAddressAndPort, Map<String,String>> loadDcRackInfo()
     {
-        Map<InetAddress, Map<String, String>> result = new HashMap<>();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, data_center, rack from system." + PEERS))
+        Map<InetAddressAndPort, Map<String, String>> result = new HashMap<>();
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, peer_port, data_center, rack from system." + PEERS_V2))
         {
-            InetAddress peer = row.getInetAddress("peer");
+            InetAddress address = row.getInetAddress("peer");
+            Integer port = row.getInt("peer_port");
+            InetAddressAndPort peer = InetAddressAndPort.getByAddressOverrideDefaults(address, port);
             if (row.has("data_center") && row.has("rack"))
             {
                 Map<String, String> dcRack = new HashMap<>();
@@ -779,16 +887,16 @@ public final class SystemKeyspace
      * @param ep endpoint address to check
      * @return Release version or null if version is unknown.
      */
-    public static CassandraVersion getReleaseVersion(InetAddress ep)
+    public static CassandraVersion getReleaseVersion(InetAddressAndPort ep)
     {
         try
         {
-            if (FBUtilities.getBroadcastAddress().equals(ep))
+            if (FBUtilities.getBroadcastAddressAndPort().equals(ep))
             {
                 return new CassandraVersion(FBUtilities.getReleaseVersionString());
             }
-            String req = "SELECT release_version FROM system.%s WHERE peer=?";
-            UntypedResultSet result = executeInternal(format(req, PEERS), ep);
+            String req = "SELECT release_version FROM system.%s WHERE peer=? AND peer_port=?";
+            UntypedResultSet result = executeInternal(String.format(req, PEERS_V2), ep.address, ep.port);
             if (result != null && result.one().has("release_version"))
             {
                 return new CassandraVersion(result.one().getString("release_version"));
@@ -1197,7 +1305,7 @@ public final class SystemKeyspace
     }
 
     public static synchronized void updateTransferredRanges(StreamOperation streamOperation,
-                                                         InetAddress peer,
+                                                         InetAddressAndPort peer,
                                                          String keyspace,
                                                          Collection<Range<Token>> streamedRanges)
     {
@@ -1207,17 +1315,21 @@ public final class SystemKeyspace
         {
             rangesToUpdate.add(rangeToBytes(range));
         }
-        executeInternal(format(cql, TRANSFERRED_RANGES), rangesToUpdate, streamOperation.getDescription(), peer, keyspace);
+        executeInternal(format(cql, LEGACY_TRANSFERRED_RANGES), rangesToUpdate, streamOperation.getDescription(), peer.address, keyspace);
+        cql = "UPDATE system.%s SET ranges = ranges + ? WHERE operation = ? AND peer = ? AND peer_port = ? AND keyspace_name = ?";
+        executeInternal(String.format(cql, TRANSFERRED_RANGES_V2), rangesToUpdate, streamOperation.getDescription(), peer.address, peer.port, keyspace);
     }
 
-    public static synchronized Map<InetAddress, Set<Range<Token>>> getTransferredRanges(String description, String keyspace, IPartitioner partitioner)
+    public static synchronized Map<InetAddressAndPort, Set<Range<Token>>> getTransferredRanges(String description, String keyspace, IPartitioner partitioner)
     {
-        Map<InetAddress, Set<Range<Token>>> result = new HashMap<>();
+        Map<InetAddressAndPort, Set<Range<Token>>> result = new HashMap<>();
         String query = "SELECT * FROM system.%s WHERE operation = ? AND keyspace_name = ?";
-        UntypedResultSet rs = executeInternal(format(query, TRANSFERRED_RANGES), description, keyspace);
+        UntypedResultSet rs = executeInternal(String.format(query, TRANSFERRED_RANGES_V2), description, keyspace);
         for (UntypedResultSet.Row row : rs)
         {
-            InetAddress peer = row.getInetAddress("peer");
+            InetAddress peerAddress = row.getInetAddress("peer");
+            int port = row.getInt("peer_port");
+            InetAddressAndPort peer = InetAddressAndPort.getByAddressOverrideDefaults(peerAddress, port);
             Set<ByteBuffer> rawRanges = row.getSet("ranges", BytesType.instance);
             Set<Range<Token>> ranges = Sets.newHashSetWithExpectedSize(rawRanges.size());
             for (ByteBuffer rawRange : rawRanges)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java b/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java
new file mode 100644
index 0000000..ea5ff59
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java
@@ -0,0 +1,184 @@
+/*
+ * 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.db;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UUIDType;
+
+/**
+ * Migrate 3.0 versions of some tables to 4.0. In this case it's just extra columns and some keys
+ * that are changed.
+ *
+ * Can't just add the additional columns because they are primary key columns and C* doesn't support changing
+ * key columns even if it's just clustering columns.
+ */
+public class SystemKeyspaceMigrator40
+{
+    static final String legacyPeersName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.LEGACY_PEERS);
+    static final String peersName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.PEERS_V2);
+    static final String legacyPeerEventsName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.LEGACY_PEER_EVENTS);
+    static final String peerEventsName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.PEER_EVENTS_V2);
+    static final String legacyTransferredRangesName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.LEGACY_TRANSFERRED_RANGES);
+    static final String transferredRangesName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.TRANSFERRED_RANGES_V2);
+
+    private static final Logger logger = LoggerFactory.getLogger(SystemKeyspaceMigrator40.class);
+
+    private SystemKeyspaceMigrator40() {}
+
+    public static void migrate()
+    {
+        migratePeers();
+        migratePeerEvents();
+        migrateTransferredRanges();
+    }
+
+    private static void migratePeers()
+    {
+        ColumnFamilyStore newPeers = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.PEERS_V2);
+
+        if (!newPeers.isEmpty())
+             return;
+
+        logger.info("{} table was empty, migrating legacy {}, if this fails you should fix the issue and then truncate {} to have it try again.",
+                                  peersName, legacyPeersName, peersName);
+
+        String query = String.format("SELECT * FROM %s",
+                                     legacyPeersName);
+
+        String insert = String.format("INSERT INTO %s ( "
+                                      + "peer, "
+                                      + "peer_port, "
+                                      + "data_center, "
+                                      + "host_id, "
+                                      + "preferred_ip, "
+                                      + "preferred_port, "
+                                      + "rack, "
+                                      + "release_version, "
+                                      + "native_address, "
+                                      + "native_port, "
+                                      + "schema_version, "
+                                      + "tokens) "
+                                      + " values ( ?, ?, ? , ? , ?, ?, ?, ?, ?, ?, ?, ?)",
+                                      peersName);
+
+        UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, 1000);
+        int transferred = 0;
+        logger.info("Migrating rows from legacy {} to {}", legacyPeersName, peersName);
+        for (UntypedResultSet.Row row : rows)
+        {
+            logger.debug("Transferring row {}", transferred);
+            QueryProcessor.executeInternal(insert,
+                                           row.has("peer") ? row.getInetAddress("peer") : null,
+                                           DatabaseDescriptor.getStoragePort(),
+                                           row.has("data_center") ? row.getString("data_center") : null,
+                                           row.has("host_id") ? row.getUUID("host_id") : null,
+                                           row.has("preferred_ip") ? row.getInetAddress("preferred_ip") : null,
+                                           DatabaseDescriptor.getStoragePort(),
+                                           row.has("rack") ? row.getString("rack") : null,
+                                           row.has("release_version") ? row.getString("release_version") : null,
+                                           row.has("rpc_address") ? row.getInetAddress("rpc_address") : null,
+                                           DatabaseDescriptor.getNativeTransportPort(),
+                                           row.has("schema_version") ? row.getUUID("schema_version") : null,
+                                           row.has("tokens") ? row.getSet("tokens", UTF8Type.instance) : null);
+            transferred++;
+        }
+        logger.info("Migrated {} rows from legacy {} to {}", transferred, legacyPeersName, peersName);
+    }
+
+    private static void migratePeerEvents()
+    {
+        ColumnFamilyStore newPeerEvents = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.PEER_EVENTS_V2);
+
+        if (!newPeerEvents.isEmpty())
+            return;
+
+        logger.info("{} table was empty, migrating legacy {} to {}", peerEventsName, legacyPeerEventsName, peerEventsName);
+
+        String query = String.format("SELECT * FROM %s",
+                                     legacyPeerEventsName);
+
+        String insert = String.format("INSERT INTO %s ( "
+                                      + "peer, "
+                                      + "peer_port, "
+                                      + "hints_dropped) "
+                                      + " values ( ?, ?, ? )",
+                                      peerEventsName);
+
+        UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, 1000);
+        int transferred = 0;
+        for (UntypedResultSet.Row row : rows)
+        {
+            logger.debug("Transferring row {}", transferred);
+            QueryProcessor.executeInternal(insert,
+                                           row.has("peer") ? row.getInetAddress("peer") : null,
+                                           DatabaseDescriptor.getStoragePort(),
+                                           row.has("hints_dropped") ? row.getMap("hints_dropped", UUIDType.instance, Int32Type.instance) : null);
+            transferred++;
+        }
+        logger.info("Migrated {} rows from legacy {} to {}", transferred, legacyPeerEventsName, peerEventsName);
+    }
+
+    static void migrateTransferredRanges()
+    {
+        ColumnFamilyStore newTransferredRanges = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.TRANSFERRED_RANGES_V2);
+
+        if (!newTransferredRanges.isEmpty())
+            return;
+
+        logger.info("{} table was empty, migrating legacy {} to {}", transferredRangesName, legacyTransferredRangesName, transferredRangesName);
+
+        String query = String.format("SELECT * FROM %s",
+                                     legacyTransferredRangesName);
+
+        String insert = String.format("INSERT INTO %s ("
+                                      + "operation, "
+                                      + "peer, "
+                                      + "peer_port, "
+                                      + "keyspace_name, "
+                                      + "ranges) "
+                                      + " values ( ?, ?, ? , ?, ?)",
+                                      transferredRangesName);
+
+        UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, 1000);
+        int transferred = 0;
+        for (UntypedResultSet.Row row : rows)
+        {
+            logger.debug("Transferring row {}", transferred);
+            QueryProcessor.executeInternal(insert,
+                                           row.has("operation") ? row.getString("operation") : null,
+                                           row.has("peer") ? row.getInetAddress("peer") : null,
+                                           DatabaseDescriptor.getStoragePort(),
+                                           row.has("keyspace_name") ? row.getString("keyspace_name") : null,
+                                           row.has("ranges") ? row.getSet("ranges", BytesType.instance) : null);
+            transferred++;
+        }
+
+        logger.info("Migrated {} rows from legacy {} to {}", transferred, legacyTransferredRangesName, transferredRangesName);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/view/ViewUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewUtils.java b/src/java/org/apache/cassandra/db/view/ViewUtils.java
index 4dc1766..df16943 100644
--- a/src/java/org/apache/cassandra/db/view/ViewUtils.java
+++ b/src/java/org/apache/cassandra/db/view/ViewUtils.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db.view;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Optional;
@@ -27,6 +26,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -58,14 +58,14 @@ public final class ViewUtils
      *
      * @return Optional.empty() if this method is called using a base token which does not belong to this replica
      */
-    public static Optional<InetAddress> getViewNaturalEndpoint(String keyspaceName, Token baseToken, Token viewToken)
+    public static Optional<InetAddressAndPort> getViewNaturalEndpoint(String keyspaceName, Token baseToken, Token viewToken)
     {
         AbstractReplicationStrategy replicationStrategy = Keyspace.open(keyspaceName).getReplicationStrategy();
 
-        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
-        List<InetAddress> baseEndpoints = new ArrayList<>();
-        List<InetAddress> viewEndpoints = new ArrayList<>();
-        for (InetAddress baseEndpoint : replicationStrategy.getNaturalEndpoints(baseToken))
+        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
+        List<InetAddressAndPort> baseEndpoints = new ArrayList<>();
+        List<InetAddressAndPort> viewEndpoints = new ArrayList<>();
+        for (InetAddressAndPort baseEndpoint : replicationStrategy.getNaturalEndpoints(baseToken))
         {
             // An endpoint is local if we're not using Net
             if (!(replicationStrategy instanceof NetworkTopologyStrategy) ||
@@ -73,10 +73,10 @@ public final class ViewUtils
                 baseEndpoints.add(baseEndpoint);
         }
 
-        for (InetAddress viewEndpoint : replicationStrategy.getNaturalEndpoints(viewToken))
+        for (InetAddressAndPort viewEndpoint : replicationStrategy.getNaturalEndpoints(viewToken))
         {
             // If we are a base endpoint which is also a view replica, we use ourselves as our view replica
-            if (viewEndpoint.equals(FBUtilities.getBroadcastAddress()))
+            if (viewEndpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
                 return Optional.of(viewEndpoint);
 
             // We have to remove any endpoint which is shared between the base and the view, as it will select itself
@@ -92,7 +92,7 @@ public final class ViewUtils
         // Since the same replication strategy is used, the same placement should be used and we should get the same
         // number of replicas for all of the tokens in the ring.
         assert baseEndpoints.size() == viewEndpoints.size() : "Replication strategy should have the same number of endpoints for the base and the view";
-        int baseIdx = baseEndpoints.indexOf(FBUtilities.getBroadcastAddress());
+        int baseIdx = baseEndpoints.indexOf(FBUtilities.getBroadcastAddressAndPort());
 
         if (baseIdx < 0)
             //This node is not a base replica of this key, so we return empty

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index a25f867..432586b 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.dht;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -38,6 +37,7 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.*;
@@ -51,12 +51,12 @@ public class BootStrapper extends ProgressEventNotifierSupport
     private static final Logger logger = LoggerFactory.getLogger(BootStrapper.class);
 
     /* endpoint that needs to be bootstrapped */
-    protected final InetAddress address;
+    protected final InetAddressAndPort address;
     /* token of the node being bootstrapped. */
     protected final Collection<Token> tokens;
     protected final TokenMetadata tokenMetadata;
 
-    public BootStrapper(InetAddress address, Collection<Token> tokens, TokenMetadata tmd)
+    public BootStrapper(InetAddressAndPort address, Collection<Token> tokens, TokenMetadata tmd)
     {
         assert address != null;
         assert tokens != null && !tokens.isEmpty();
@@ -159,7 +159,7 @@ public class BootStrapper extends ProgressEventNotifierSupport
      * otherwise, if allocationKeyspace is specified use the token allocation algorithm to generate suitable tokens
      * else choose num_tokens tokens at random
      */
-    public static Collection<Token> getBootstrapTokens(final TokenMetadata metadata, InetAddress address, int schemaWaitDelay) throws ConfigurationException
+    public static Collection<Token> getBootstrapTokens(final TokenMetadata metadata, InetAddressAndPort address, int schemaWaitDelay) throws ConfigurationException
     {
         String allocationKeyspace = DatabaseDescriptor.getAllocateTokensForKeyspace();
         Collection<String> initialTokens = DatabaseDescriptor.getInitialTokens();
@@ -199,13 +199,13 @@ public class BootStrapper extends ProgressEventNotifierSupport
     }
 
     static Collection<Token> allocateTokens(final TokenMetadata metadata,
-                                            InetAddress address,
+                                            InetAddressAndPort address,
                                             String allocationKeyspace,
                                             int numTokens,
                                             int schemaWaitDelay)
     {
         StorageService.instance.waitForSchema(schemaWaitDelay);
-        if (!FBUtilities.getBroadcastAddress().equals(InetAddress.getLoopbackAddress()))
+        if (!FBUtilities.getBroadcastAddressAndPort().equals(InetAddressAndPort.getLoopbackAddress()))
             Gossiper.waitToSettle();
 
         Keyspace ks = Keyspace.open(allocationKeyspace);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java b/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java
index d407212..b90bc96 100644
--- a/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java
+++ b/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java
@@ -32,6 +32,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.FBUtilities;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -72,7 +73,7 @@ public class RangeFetchMapCalculator
 {
     private static final Logger logger = LoggerFactory.getLogger(RangeFetchMapCalculator.class);
     private static final long TRIVIAL_RANGE_LIMIT = 1000;
-    private final Multimap<Range<Token>, InetAddress> rangesWithSources;
+    private final Multimap<Range<Token>, InetAddressAndPort> rangesWithSources;
     private final Collection<RangeStreamer.ISourceFilter> sourceFilters;
     private final String keyspace;
     //We need two Vertices to act as source and destination in the algorithm
@@ -80,7 +81,7 @@ public class RangeFetchMapCalculator
     private final Vertex destinationVertex = OuterVertex.getDestinationVertex();
     private final Set<Range<Token>> trivialRanges;
 
-    public RangeFetchMapCalculator(Multimap<Range<Token>, InetAddress> rangesWithSources,
+    public RangeFetchMapCalculator(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources,
                                    Collection<RangeStreamer.ISourceFilter> sourceFilters,
                                    String keyspace)
     {
@@ -108,16 +109,16 @@ public class RangeFetchMapCalculator
         return false;
     }
 
-    public Multimap<InetAddress, Range<Token>> getRangeFetchMap()
+    public Multimap<InetAddressAndPort, Range<Token>> getRangeFetchMap()
     {
-        Multimap<InetAddress, Range<Token>> fetchMap = HashMultimap.create();
+        Multimap<InetAddressAndPort, Range<Token>> fetchMap = HashMultimap.create();
         fetchMap.putAll(getRangeFetchMapForNonTrivialRanges());
         fetchMap.putAll(getRangeFetchMapForTrivialRanges(fetchMap));
         return fetchMap;
     }
 
     @VisibleForTesting
-    Multimap<InetAddress, Range<Token>> getRangeFetchMapForNonTrivialRanges()
+    Multimap<InetAddressAndPort, Range<Token>> getRangeFetchMapForNonTrivialRanges()
     {
         //Get the graph with edges between ranges and their source endpoints
         MutableCapacityGraph<Vertex, Integer> graph = getGraph();
@@ -148,19 +149,19 @@ public class RangeFetchMapCalculator
     }
 
     @VisibleForTesting
-    Multimap<InetAddress, Range<Token>> getRangeFetchMapForTrivialRanges(Multimap<InetAddress, Range<Token>> optimisedMap)
+    Multimap<InetAddressAndPort, Range<Token>> getRangeFetchMapForTrivialRanges(Multimap<InetAddressAndPort, Range<Token>> optimisedMap)
     {
-        Multimap<InetAddress, Range<Token>> fetchMap = HashMultimap.create();
+        Multimap<InetAddressAndPort, Range<Token>> fetchMap = HashMultimap.create();
         for (Range<Token> trivialRange : trivialRanges)
         {
             boolean added = false;
             boolean localDCCheck = true;
             while (!added)
             {
-                List<InetAddress> srcs = new ArrayList<>(rangesWithSources.get(trivialRange));
+                List<InetAddressAndPort> srcs = new ArrayList<>(rangesWithSources.get(trivialRange));
                 // sort with the endpoint having the least number of streams first:
                 srcs.sort(Comparator.comparingInt(o -> optimisedMap.get(o).size()));
-                for (InetAddress src : srcs)
+                for (InetAddressAndPort src : srcs)
                 {
                     if (passFilters(src, localDCCheck))
                     {
@@ -202,9 +203,9 @@ public class RangeFetchMapCalculator
      * @param result Flow algorithm result
      * @return  Multi Map of Machine to Ranges
      */
-    private Multimap<InetAddress, Range<Token>> getRangeFetchMapFromGraphResult(MutableCapacityGraph<Vertex, Integer> graph, MaximumFlowAlgorithmResult<Integer, CapacityEdge<Vertex, Integer>> result)
+    private Multimap<InetAddressAndPort, Range<Token>> getRangeFetchMapFromGraphResult(MutableCapacityGraph<Vertex, Integer> graph, MaximumFlowAlgorithmResult<Integer, CapacityEdge<Vertex, Integer>> result)
     {
-        final Multimap<InetAddress, Range<Token>> rangeFetchMapMap = HashMultimap.create();
+        final Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap = HashMultimap.create();
         if(result == null)
             return rangeFetchMapMap;
         final Function<CapacityEdge<Vertex, Integer>, Integer> flowFunction = result.calcFlowFunction();
@@ -346,13 +347,13 @@ public class RangeFetchMapCalculator
     private boolean addEndpoints(MutableCapacityGraph<Vertex, Integer> capacityGraph, RangeVertex rangeVertex, boolean localDCCheck)
     {
         boolean sourceFound = false;
-        for (InetAddress endpoint : rangesWithSources.get(rangeVertex.getRange()))
+        for (InetAddressAndPort endpoint : rangesWithSources.get(rangeVertex.getRange()))
         {
             if (passFilters(endpoint, localDCCheck))
             {
                 sourceFound = true;
                 // if we pass filters, it means that we don't filter away localhost and we can count it as a source:
-                if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+                if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
                     continue; // but don't add localhost to the graph to avoid streaming locally
                 final Vertex endpointVertex = new EndpointVertex(endpoint);
                 capacityGraph.insertVertex(rangeVertex);
@@ -363,7 +364,7 @@ public class RangeFetchMapCalculator
         return sourceFound;
     }
 
-    private boolean isInLocalDC(InetAddress endpoint)
+    private boolean isInLocalDC(InetAddressAndPort endpoint)
     {
         return DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint));
     }
@@ -374,7 +375,7 @@ public class RangeFetchMapCalculator
      * @param localDCCheck Allow endpoints with local DC
      * @return   True if filters pass this endpoint
      */
-    private boolean passFilters(final InetAddress endpoint, boolean localDCCheck)
+    private boolean passFilters(final InetAddressAndPort endpoint, boolean localDCCheck)
     {
         for (RangeStreamer.ISourceFilter filter : sourceFilters)
         {
@@ -410,15 +411,15 @@ public class RangeFetchMapCalculator
      */
     private static class EndpointVertex extends Vertex
     {
-        private final InetAddress endpoint;
+        private final InetAddressAndPort endpoint;
 
-        public EndpointVertex(InetAddress endpoint)
+        public EndpointVertex(InetAddressAndPort endpoint)
         {
             assert endpoint != null;
             this.endpoint = endpoint;
         }
 
-        public InetAddress getEndpoint()
+        public InetAddressAndPort getEndpoint()
         {
             return endpoint;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/dht/RangeStreamer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java
index eabb212..439ebc6 100644
--- a/src/java/org/apache/cassandra/dht/RangeStreamer.java
+++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.dht;
 
-import java.net.InetAddress;
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -26,8 +25,9 @@ import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.LocalStrategy;
-import org.apache.cassandra.service.ActiveRepairService;
+
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -59,10 +59,10 @@ public class RangeStreamer
     /* current token ring */
     private final TokenMetadata metadata;
     /* address of this node */
-    private final InetAddress address;
+    private final InetAddressAndPort address;
     /* streaming description */
     private final String description;
-    private final Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = HashMultimap.create();
+    private final Multimap<String, Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> toFetch = HashMultimap.create();
     private final Set<ISourceFilter> sourceFilters = new HashSet<>();
     private final StreamPlan streamPlan;
     private final boolean useStrictConsistency;
@@ -74,7 +74,7 @@ public class RangeStreamer
      */
     public static interface ISourceFilter
     {
-        public boolean shouldInclude(InetAddress endpoint);
+        public boolean shouldInclude(InetAddressAndPort endpoint);
     }
 
     /**
@@ -90,7 +90,7 @@ public class RangeStreamer
             this.fd = fd;
         }
 
-        public boolean shouldInclude(InetAddress endpoint)
+        public boolean shouldInclude(InetAddressAndPort endpoint)
         {
             return fd.isAlive(endpoint);
         }
@@ -110,7 +110,7 @@ public class RangeStreamer
             this.snitch = snitch;
         }
 
-        public boolean shouldInclude(InetAddress endpoint)
+        public boolean shouldInclude(InetAddressAndPort endpoint)
         {
             return snitch.getDatacenter(endpoint).equals(sourceDc);
         }
@@ -121,9 +121,9 @@ public class RangeStreamer
      */
     public static class ExcludeLocalNodeFilter implements ISourceFilter
     {
-        public boolean shouldInclude(InetAddress endpoint)
+        public boolean shouldInclude(InetAddressAndPort endpoint)
         {
-            return !FBUtilities.getBroadcastAddress().equals(endpoint);
+            return !FBUtilities.getBroadcastAddressAndPort().equals(endpoint);
         }
     }
 
@@ -132,14 +132,14 @@ public class RangeStreamer
      */
     public static class WhitelistedSourcesFilter implements ISourceFilter
     {
-        private final Set<InetAddress> whitelistedSources;
+        private final Set<InetAddressAndPort> whitelistedSources;
 
-        public WhitelistedSourcesFilter(Set<InetAddress> whitelistedSources)
+        public WhitelistedSourcesFilter(Set<InetAddressAndPort> whitelistedSources)
         {
             this.whitelistedSources = whitelistedSources;
         }
 
-        public boolean shouldInclude(InetAddress endpoint)
+        public boolean shouldInclude(InetAddressAndPort endpoint)
         {
             return whitelistedSources.contains(endpoint);
         }
@@ -147,7 +147,7 @@ public class RangeStreamer
 
     public RangeStreamer(TokenMetadata metadata,
                          Collection<Token> tokens,
-                         InetAddress address,
+                         InetAddressAndPort address,
                          StreamOperation streamOperation,
                          boolean useStrictConsistency,
                          IEndpointSnitch snitch,
@@ -186,18 +186,18 @@ public class RangeStreamer
         }
 
         boolean useStrictSource = useStrictSourcesForRanges(keyspaceName);
-        Multimap<Range<Token>, InetAddress> rangesForKeyspace = useStrictSource
+        Multimap<Range<Token>, InetAddressAndPort> rangesForKeyspace = useStrictSource
                 ? getAllRangesWithStrictSourcesFor(keyspaceName, ranges) : getAllRangesWithSourcesFor(keyspaceName, ranges);
 
-        for (Map.Entry<Range<Token>, InetAddress> entry : rangesForKeyspace.entries())
+        for (Map.Entry<Range<Token>, InetAddressAndPort> entry : rangesForKeyspace.entries())
             logger.info("{}: range {} exists on {} for keyspace {}", description, entry.getKey(), entry.getValue(), keyspaceName);
 
         AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
-        Multimap<InetAddress, Range<Token>> rangeFetchMap = useStrictSource || strat == null || strat.getReplicationFactor() == 1
+        Multimap<InetAddressAndPort, Range<Token>> rangeFetchMap = useStrictSource || strat == null || strat.getReplicationFactor() == 1
                                                             ? getRangeFetchMap(rangesForKeyspace, sourceFilters, keyspaceName, useStrictConsistency)
                                                             : getOptimizedRangeFetchMap(rangesForKeyspace, sourceFilters, keyspaceName);
 
-        for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : rangeFetchMap.asMap().entrySet())
+        for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : rangeFetchMap.asMap().entrySet())
         {
             if (logger.isTraceEnabled())
             {
@@ -226,19 +226,19 @@ public class RangeStreamer
      *
      * @throws java.lang.IllegalStateException when there is no source to get data streamed
      */
-    private Multimap<Range<Token>, InetAddress> getAllRangesWithSourcesFor(String keyspaceName, Collection<Range<Token>> desiredRanges)
+    private Multimap<Range<Token>, InetAddressAndPort> getAllRangesWithSourcesFor(String keyspaceName, Collection<Range<Token>> desiredRanges)
     {
         AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
-        Multimap<Range<Token>, InetAddress> rangeAddresses = strat.getRangeAddresses(metadata.cloneOnlyTokenMap());
+        Multimap<Range<Token>, InetAddressAndPort> rangeAddresses = strat.getRangeAddresses(metadata.cloneOnlyTokenMap());
 
-        Multimap<Range<Token>, InetAddress> rangeSources = ArrayListMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangeSources = ArrayListMultimap.create();
         for (Range<Token> desiredRange : desiredRanges)
         {
             for (Range<Token> range : rangeAddresses.keySet())
             {
                 if (range.contains(desiredRange))
                 {
-                    List<InetAddress> preferred = snitch.getSortedListByProximity(address, rangeAddresses.get(range));
+                    List<InetAddressAndPort> preferred = snitch.getSortedListByProximity(address, rangeAddresses.get(range));
                     rangeSources.putAll(desiredRange, preferred);
                     break;
                 }
@@ -258,30 +258,30 @@ public class RangeStreamer
      *
      * @throws java.lang.IllegalStateException when there is no source to get data streamed, or more than 1 source found.
      */
-    private Multimap<Range<Token>, InetAddress> getAllRangesWithStrictSourcesFor(String keyspace, Collection<Range<Token>> desiredRanges)
+    private Multimap<Range<Token>, InetAddressAndPort> getAllRangesWithStrictSourcesFor(String keyspace, Collection<Range<Token>> desiredRanges)
     {
         assert tokens != null;
         AbstractReplicationStrategy strat = Keyspace.open(keyspace).getReplicationStrategy();
 
         // Active ranges
         TokenMetadata metadataClone = metadata.cloneOnlyTokenMap();
-        Multimap<Range<Token>, InetAddress> addressRanges = strat.getRangeAddresses(metadataClone);
+        Multimap<Range<Token>, InetAddressAndPort> addressRanges = strat.getRangeAddresses(metadataClone);
 
         // Pending ranges
         metadataClone.updateNormalTokens(tokens, address);
-        Multimap<Range<Token>, InetAddress> pendingRangeAddresses = strat.getRangeAddresses(metadataClone);
+        Multimap<Range<Token>, InetAddressAndPort> pendingRangeAddresses = strat.getRangeAddresses(metadataClone);
 
         // Collects the source that will have its range moved to the new node
-        Multimap<Range<Token>, InetAddress> rangeSources = ArrayListMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangeSources = ArrayListMultimap.create();
 
         for (Range<Token> desiredRange : desiredRanges)
         {
-            for (Map.Entry<Range<Token>, Collection<InetAddress>> preEntry : addressRanges.asMap().entrySet())
+            for (Map.Entry<Range<Token>, Collection<InetAddressAndPort>> preEntry : addressRanges.asMap().entrySet())
             {
                 if (preEntry.getKey().contains(desiredRange))
                 {
-                    Set<InetAddress> oldEndpoints = Sets.newHashSet(preEntry.getValue());
-                    Set<InetAddress> newEndpoints = Sets.newHashSet(pendingRangeAddresses.get(desiredRange));
+                    Set<InetAddressAndPort> oldEndpoints = Sets.newHashSet(preEntry.getValue());
+                    Set<InetAddressAndPort> newEndpoints = Sets.newHashSet(pendingRangeAddresses.get(desiredRange));
 
                     // Due to CASSANDRA-5953 we can have a higher RF then we have endpoints.
                     // So we need to be careful to only be strict when endpoints == RF
@@ -296,14 +296,14 @@ public class RangeStreamer
             }
 
             // Validate
-            Collection<InetAddress> addressList = rangeSources.get(desiredRange);
+            Collection<InetAddressAndPort> addressList = rangeSources.get(desiredRange);
             if (addressList == null || addressList.isEmpty())
                 throw new IllegalStateException("No sources found for " + desiredRange);
 
             if (addressList.size() > 1)
                 throw new IllegalStateException("Multiple endpoints found for " + desiredRange);
 
-            InetAddress sourceIp = addressList.iterator().next();
+            InetAddressAndPort sourceIp = addressList.iterator().next();
             EndpointState sourceState = Gossiper.instance.getEndpointStateForEndpoint(sourceIp);
             if (Gossiper.instance.isEnabled() && (sourceState == null || !sourceState.isAlive()))
                 throw new RuntimeException("A node required to move the data consistently is down (" + sourceIp + "). " +
@@ -320,17 +320,17 @@ public class RangeStreamer
      * @param keyspace keyspace name
      * @return Map of source endpoint to collection of ranges
      */
-    private static Multimap<InetAddress, Range<Token>> getRangeFetchMap(Multimap<Range<Token>, InetAddress> rangesWithSources,
-                                                                        Collection<ISourceFilter> sourceFilters, String keyspace,
-                                                                        boolean useStrictConsistency)
+    private static Multimap<InetAddressAndPort, Range<Token>> getRangeFetchMap(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources,
+                                                                               Collection<ISourceFilter> sourceFilters, String keyspace,
+                                                                               boolean useStrictConsistency)
     {
-        Multimap<InetAddress, Range<Token>> rangeFetchMapMap = HashMultimap.create();
+        Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap = HashMultimap.create();
         for (Range<Token> range : rangesWithSources.keySet())
         {
             boolean foundSource = false;
 
             outer:
-            for (InetAddress address : rangesWithSources.get(range))
+            for (InetAddressAndPort address : rangesWithSources.get(range))
             {
                 for (ISourceFilter filter : sourceFilters)
                 {
@@ -338,7 +338,7 @@ public class RangeStreamer
                         continue outer;
                 }
 
-                if (address.equals(FBUtilities.getBroadcastAddress()))
+                if (address.equals(FBUtilities.getBroadcastAddressAndPort()))
                 {
                     // If localhost is a source, we have found one, but we don't add it to the map to avoid streaming locally
                     foundSource = true;
@@ -371,11 +371,11 @@ public class RangeStreamer
     }
 
 
-    private static Multimap<InetAddress, Range<Token>> getOptimizedRangeFetchMap(Multimap<Range<Token>, InetAddress> rangesWithSources,
-                                                                        Collection<ISourceFilter> sourceFilters, String keyspace)
+    private static Multimap<InetAddressAndPort, Range<Token>> getOptimizedRangeFetchMap(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources,
+                                                                                        Collection<ISourceFilter> sourceFilters, String keyspace)
     {
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, sourceFilters, keyspace);
-        Multimap<InetAddress, Range<Token>> rangeFetchMapMap = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap = calculator.getRangeFetchMap();
         logger.info("Output from RangeFetchMapCalculator for keyspace {}", keyspace);
         validateRangeFetchMap(rangesWithSources, rangeFetchMapMap, keyspace);
         return rangeFetchMapMap;
@@ -387,11 +387,11 @@ public class RangeStreamer
      * @param rangeFetchMapMap
      * @param keyspace
      */
-    private static void validateRangeFetchMap(Multimap<Range<Token>, InetAddress> rangesWithSources, Multimap<InetAddress, Range<Token>> rangeFetchMapMap, String keyspace)
+    private static void validateRangeFetchMap(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources, Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap, String keyspace)
     {
-        for (Map.Entry<InetAddress, Range<Token>> entry : rangeFetchMapMap.entries())
+        for (Map.Entry<InetAddressAndPort, Range<Token>> entry : rangeFetchMapMap.entries())
         {
-            if(entry.getKey().equals(FBUtilities.getBroadcastAddress()))
+            if(entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
             {
                 throw new IllegalStateException("Trying to stream locally. Range: " + entry.getValue()
                                         + " in keyspace " + keyspace);
@@ -407,26 +407,26 @@ public class RangeStreamer
         }
     }
 
-    public static Multimap<InetAddress, Range<Token>> getWorkMap(Multimap<Range<Token>, InetAddress> rangesWithSourceTarget, String keyspace,
-                                                                 IFailureDetector fd, boolean useStrictConsistency)
+    public static Multimap<InetAddressAndPort, Range<Token>> getWorkMap(Multimap<Range<Token>, InetAddressAndPort> rangesWithSourceTarget, String keyspace,
+                                                                        IFailureDetector fd, boolean useStrictConsistency)
     {
         return getRangeFetchMap(rangesWithSourceTarget, Collections.<ISourceFilter>singleton(new FailureDetectorSourceFilter(fd)), keyspace, useStrictConsistency);
     }
 
     // For testing purposes
     @VisibleForTesting
-    Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch()
+    Multimap<String, Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> toFetch()
     {
         return toFetch;
     }
 
     public StreamResultFuture fetchAsync()
     {
-        for (Map.Entry<String, Map.Entry<InetAddress, Collection<Range<Token>>>> entry : toFetch.entries())
+        for (Map.Entry<String, Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> entry : toFetch.entries())
         {
             String keyspace = entry.getKey();
-            InetAddress source = entry.getValue().getKey();
-            InetAddress preferred = SystemKeyspace.getPreferredIP(source);
+            InetAddressAndPort source = entry.getValue().getKey();
+            InetAddressAndPort preferred = SystemKeyspace.getPreferredIP(source);
             Collection<Range<Token>> ranges = entry.getValue().getValue();
 
             // filter out already streamed ranges

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
index 7c9d22c..61082df 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.dht.tokenallocator;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
@@ -37,6 +36,7 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.locator.TokenMetadata;
@@ -48,7 +48,7 @@ public class TokenAllocation
 
     public static Collection<Token> allocateTokens(final TokenMetadata tokenMetadata,
                                                    final AbstractReplicationStrategy rs,
-                                                   final InetAddress endpoint,
+                                                   final InetAddressAndPort endpoint,
                                                    int numTokens)
     {
         TokenMetadata tokenMetadataCopy = tokenMetadata.cloneOnlyTokenMap();
@@ -81,7 +81,7 @@ public class TokenAllocation
         {
             while (tokenMetadata.getEndpoint(t) != null)
             {
-                InetAddress other = tokenMetadata.getEndpoint(t);
+                InetAddressAndPort other = tokenMetadata.getEndpoint(t);
                 if (strategy.inAllocationRing(other))
                     throw new ConfigurationException(String.format("Allocated token %s already assigned to node %s. Is another node also allocating tokens?", t, other));
                 t = t.increaseSlightly();
@@ -92,9 +92,9 @@ public class TokenAllocation
     }
 
     // return the ratio of ownership for each endpoint
-    public static Map<InetAddress, Double> evaluateReplicatedOwnership(TokenMetadata tokenMetadata, AbstractReplicationStrategy rs)
+    public static Map<InetAddressAndPort, Double> evaluateReplicatedOwnership(TokenMetadata tokenMetadata, AbstractReplicationStrategy rs)
     {
-        Map<InetAddress, Double> ownership = Maps.newHashMap();
+        Map<InetAddressAndPort, Double> ownership = Maps.newHashMap();
         List<Token> sortedTokens = tokenMetadata.sortedTokens();
         Iterator<Token> it = sortedTokens.iterator();
         Token current = it.next();
@@ -109,11 +109,11 @@ public class TokenAllocation
         return ownership;
     }
 
-    static void addOwnership(final TokenMetadata tokenMetadata, final AbstractReplicationStrategy rs, Token current, Token next, Map<InetAddress, Double> ownership)
+    static void addOwnership(final TokenMetadata tokenMetadata, final AbstractReplicationStrategy rs, Token current, Token next, Map<InetAddressAndPort, Double> ownership)
     {
         double size = current.size(next);
         Token representative = current.getPartitioner().midpoint(current, next);
-        for (InetAddress n : rs.calculateNaturalEndpoints(representative, tokenMetadata))
+        for (InetAddressAndPort n : rs.calculateNaturalEndpoints(representative, tokenMetadata))
         {
             Double v = ownership.get(n);
             ownership.put(n, v != null ? v + size : size);
@@ -126,11 +126,11 @@ public class TokenAllocation
     }
 
     public static SummaryStatistics replicatedOwnershipStats(TokenMetadata tokenMetadata,
-                                                             AbstractReplicationStrategy rs, InetAddress endpoint)
+                                                             AbstractReplicationStrategy rs, InetAddressAndPort endpoint)
     {
         SummaryStatistics stat = new SummaryStatistics();
         StrategyAdapter strategy = getStrategy(tokenMetadata, rs, endpoint);
-        for (Map.Entry<InetAddress, Double> en : evaluateReplicatedOwnership(tokenMetadata, rs).entrySet())
+        for (Map.Entry<InetAddressAndPort, Double> en : evaluateReplicatedOwnership(tokenMetadata, rs).entrySet())
         {
             // Filter only in the same datacentre.
             if (strategy.inAllocationRing(en.getKey()))
@@ -139,10 +139,10 @@ public class TokenAllocation
         return stat;
     }
 
-    static TokenAllocator<InetAddress> create(TokenMetadata tokenMetadata, StrategyAdapter strategy)
+    static TokenAllocator<InetAddressAndPort> create(TokenMetadata tokenMetadata, StrategyAdapter strategy)
     {
-        NavigableMap<Token, InetAddress> sortedTokens = new TreeMap<>();
-        for (Map.Entry<Token, InetAddress> en : tokenMetadata.getNormalAndBootstrappingTokenToEndpointMap().entrySet())
+        NavigableMap<Token, InetAddressAndPort> sortedTokens = new TreeMap<>();
+        for (Map.Entry<Token, InetAddressAndPort> en : tokenMetadata.getNormalAndBootstrappingTokenToEndpointMap().entrySet())
         {
             if (strategy.inAllocationRing(en.getValue()))
                 sortedTokens.put(en.getKey(), en.getValue());
@@ -150,15 +150,15 @@ public class TokenAllocation
         return TokenAllocatorFactory.createTokenAllocator(sortedTokens, strategy, tokenMetadata.partitioner);
     }
 
-    interface StrategyAdapter extends ReplicationStrategy<InetAddress>
+    interface StrategyAdapter extends ReplicationStrategy<InetAddressAndPort>
     {
         // return true iff the provided endpoint occurs in the same virtual token-ring we are allocating for
         // i.e. the set of the nodes that share ownership with the node we are allocating
         // alternatively: return false if the endpoint's ownership is independent of the node we are allocating tokens for
-        boolean inAllocationRing(InetAddress other);
+        boolean inAllocationRing(InetAddressAndPort other);
     }
 
-    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final AbstractReplicationStrategy rs, final InetAddress endpoint)
+    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final AbstractReplicationStrategy rs, final InetAddressAndPort endpoint)
     {
         if (rs instanceof NetworkTopologyStrategy)
             return getStrategy(tokenMetadata, (NetworkTopologyStrategy) rs, rs.snitch, endpoint);
@@ -167,7 +167,7 @@ public class TokenAllocation
         throw new ConfigurationException("Token allocation does not support replication strategy " + rs.getClass().getSimpleName());
     }
 
-    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final SimpleStrategy rs, final InetAddress endpoint)
+    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final SimpleStrategy rs, final InetAddressAndPort endpoint)
     {
         final int replicas = rs.getReplicationFactor();
 
@@ -180,20 +180,20 @@ public class TokenAllocation
             }
 
             @Override
-            public Object getGroup(InetAddress unit)
+            public Object getGroup(InetAddressAndPort unit)
             {
                 return unit;
             }
 
             @Override
-            public boolean inAllocationRing(InetAddress other)
+            public boolean inAllocationRing(InetAddressAndPort other)
             {
                 return true;
             }
         };
     }
 
-    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final NetworkTopologyStrategy rs, final IEndpointSnitch snitch, final InetAddress endpoint)
+    static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final NetworkTopologyStrategy rs, final IEndpointSnitch snitch, final InetAddressAndPort endpoint)
     {
         final String dc = snitch.getDatacenter(endpoint);
         final int replicas = rs.getReplicationFactor(dc);
@@ -210,13 +210,13 @@ public class TokenAllocation
                 }
 
                 @Override
-                public Object getGroup(InetAddress unit)
+                public Object getGroup(InetAddressAndPort unit)
                 {
                     return unit;
                 }
 
                 @Override
-                public boolean inAllocationRing(InetAddress other)
+                public boolean inAllocationRing(InetAddressAndPort other)
                 {
                     return dc.equals(snitch.getDatacenter(other));
                 }
@@ -237,13 +237,13 @@ public class TokenAllocation
                 }
 
                 @Override
-                public Object getGroup(InetAddress unit)
+                public Object getGroup(InetAddressAndPort unit)
                 {
                     return snitch.getRack(unit);
                 }
 
                 @Override
-                public boolean inAllocationRing(InetAddress other)
+                public boolean inAllocationRing(InetAddressAndPort other)
                 {
                     return dc.equals(snitch.getDatacenter(other));
                 }
@@ -261,13 +261,13 @@ public class TokenAllocation
                 }
 
                 @Override
-                public Object getGroup(InetAddress unit)
+                public Object getGroup(InetAddressAndPort unit)
                 {
                     return unit;
                 }
 
                 @Override
-                public boolean inAllocationRing(InetAddress other)
+                public boolean inAllocationRing(InetAddressAndPort other)
                 {
                     return dc.equals(snitch.getDatacenter(other));
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorFactory.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorFactory.java
index 58acb56..5fdba02 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorFactory.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorFactory.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.dht.tokenallocator;
 
-import java.net.InetAddress;
 import java.util.NavigableMap;
 
 import org.slf4j.Logger;
@@ -26,13 +25,14 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public class TokenAllocatorFactory
 {
     private static final Logger logger = LoggerFactory.getLogger(TokenAllocatorFactory.class);
-    public static TokenAllocator<InetAddress> createTokenAllocator(NavigableMap<Token, InetAddress> sortedTokens,
-                                                     ReplicationStrategy<InetAddress> strategy,
-                                                     IPartitioner partitioner)
+    public static TokenAllocator<InetAddressAndPort> createTokenAllocator(NavigableMap<Token, InetAddressAndPort> sortedTokens,
+                                                                          ReplicationStrategy<InetAddressAndPort> strategy,
+                                                                          IPartitioner partitioner)
     {
         if(strategy.replicas() == 1)
         {


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


[02/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 2ef9446..e57ab94 100644
--- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
+++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.repair.messages;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
@@ -44,6 +43,7 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.NodePair;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -148,9 +148,9 @@ public class RepairMessageSerializationsTest
     @Test
     public void syncRequestMessage() throws IOException
     {
-        InetAddress initiator = InetAddress.getByName("127.0.0.1");
-        InetAddress src = InetAddress.getByName("127.0.0.2");
-        InetAddress dst = InetAddress.getByName("127.0.0.3");
+        InetAddressAndPort initiator = InetAddressAndPort.getByName("127.0.0.1");
+        InetAddressAndPort src = InetAddressAndPort.getByName("127.0.0.2");
+        InetAddressAndPort dst = InetAddressAndPort.getByName("127.0.0.3");
 
         SyncRequest msg = new SyncRequest(buildRepairJobDesc(), initiator, src, dst, buildTokenRanges(), PreviewKind.NONE);
         serializeRoundTrip(msg, SyncRequest.serializer);
@@ -159,8 +159,8 @@ public class RepairMessageSerializationsTest
     @Test
     public void syncCompleteMessage() throws IOException
     {
-        InetAddress src = InetAddress.getByName("127.0.0.2");
-        InetAddress dst = InetAddress.getByName("127.0.0.3");
+        InetAddressAndPort src = InetAddressAndPort.getByName("127.0.0.2");
+        InetAddressAndPort dst = InetAddressAndPort.getByName("127.0.0.3");
         List<SessionSummary> summaries = new ArrayList<>();
         summaries.add(new SessionSummary(src, dst,
                                          Lists.newArrayList(new StreamSummary(TableId.fromUUID(UUIDGen.getTimeUUID()), 5, 100)),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java
index 9789b38..d876139 100644
--- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java
+++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.repair.messages;
 
 import java.io.IOException;
-import java.net.InetAddress;
 
 import com.google.common.collect.Sets;
 import org.junit.Assert;
@@ -27,6 +26,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.UUIDGen;
 
@@ -58,10 +58,10 @@ public class RepairMessageSerializerTest
     @Test
     public void prepareConsistentRequest() throws Exception
     {
-        InetAddress coordinator = InetAddress.getByName("10.0.0.1");
-        InetAddress peer1 = InetAddress.getByName("10.0.0.2");
-        InetAddress peer2 = InetAddress.getByName("10.0.0.3");
-        InetAddress peer3 = InetAddress.getByName("10.0.0.4");
+        InetAddressAndPort coordinator = InetAddressAndPort.getByName("10.0.0.1");
+        InetAddressAndPort peer1 = InetAddressAndPort.getByName("10.0.0.2");
+        InetAddressAndPort peer2 = InetAddressAndPort.getByName("10.0.0.3");
+        InetAddressAndPort peer3 = InetAddressAndPort.getByName("10.0.0.4");
         RepairMessage expected = new PrepareConsistentRequest(UUIDGen.getTimeUUID(),
                                                               coordinator,
                                                               Sets.newHashSet(peer1, peer2, peer3));
@@ -73,7 +73,7 @@ public class RepairMessageSerializerTest
     public void prepareConsistentResponse() throws Exception
     {
         RepairMessage expected = new PrepareConsistentResponse(UUIDGen.getTimeUUID(),
-                                                               InetAddress.getByName("10.0.0.2"),
+                                                               InetAddressAndPort.getByName("10.0.0.2"),
                                                                true);
         RepairMessage actual = serdes(expected);
         Assert.assertEquals(expected, actual);
@@ -99,7 +99,7 @@ public class RepairMessageSerializerTest
     public void finalizePromise() throws Exception
     {
         RepairMessage expected = new FinalizePromise(UUIDGen.getTimeUUID(),
-                                                     InetAddress.getByName("10.0.0.2"),
+                                                     InetAddressAndPort.getByName("10.0.0.2"),
                                                      true);
         RepairMessage actual = serdes(expected);
         Assert.assertEquals(expected, actual);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
index cbacaec..76f915e 100644
--- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
+++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
@@ -18,7 +18,6 @@
 */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.*;
 
 import javax.xml.crypto.Data;
@@ -43,6 +42,7 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -69,7 +69,7 @@ public class ActiveRepairServiceTest
 
     public String cfname;
     public ColumnFamilyStore store;
-    public InetAddress LOCAL, REMOTE;
+    public InetAddressAndPort LOCAL, REMOTE;
 
     private boolean initialized;
 
@@ -91,9 +91,9 @@ public class ActiveRepairServiceTest
             SchemaLoader.startGossiper();
             initialized = true;
 
-            LOCAL = FBUtilities.getBroadcastAddress();
+            LOCAL = FBUtilities.getBroadcastAddressAndPort();
             // generate a fake endpoint for which we can spoof receiving/sending trees
-            REMOTE = InetAddress.getByName("127.0.0.2");
+            REMOTE = InetAddressAndPort.getByName("127.0.0.2");
         }
 
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
@@ -107,10 +107,10 @@ public class ActiveRepairServiceTest
     public void testGetNeighborsPlusOne() throws Throwable
     {
         // generate rf+1 nodes, and ensure that all nodes are returned
-        Set<InetAddress> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
-        expected.remove(FBUtilities.getBroadcastAddress());
+        Set<InetAddressAndPort> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        expected.remove(FBUtilities.getBroadcastAddressAndPort());
         Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
-        Set<InetAddress> neighbors = new HashSet<>();
+        Set<InetAddressAndPort> neighbors = new HashSet<>();
         for (Range<Token> range : ranges)
         {
             neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, null, null));
@@ -126,14 +126,14 @@ public class ActiveRepairServiceTest
         // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
         addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
         AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
-        Set<InetAddress> expected = new HashSet<>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
+        Set<InetAddressAndPort> expected = new HashSet<>();
+        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddressAndPort()))
         {
             expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
         }
-        expected.remove(FBUtilities.getBroadcastAddress());
+        expected.remove(FBUtilities.getBroadcastAddressAndPort());
         Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
-        Set<InetAddress> neighbors = new HashSet<>();
+        Set<InetAddressAndPort> neighbors = new HashSet<>();
         for (Range<Token> range : ranges)
         {
             neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, null, null));
@@ -147,15 +147,15 @@ public class ActiveRepairServiceTest
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         // generate rf+1 nodes, and ensure that all nodes are returned
-        Set<InetAddress> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
-        expected.remove(FBUtilities.getBroadcastAddress());
+        Set<InetAddressAndPort> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        expected.remove(FBUtilities.getBroadcastAddressAndPort());
         // remove remote endpoints
         TokenMetadata.Topology topology = tmd.cloneOnlyTokenMap().getTopology();
-        HashSet<InetAddress> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
+        HashSet<InetAddressAndPort> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
         expected = Sets.intersection(expected, localEndpoints);
 
         Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
-        Set<InetAddress> neighbors = new HashSet<>();
+        Set<InetAddressAndPort> neighbors = new HashSet<>();
         for (Range<Token> range : ranges)
         {
             neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null));
@@ -171,19 +171,19 @@ public class ActiveRepairServiceTest
         // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
         addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
         AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
-        Set<InetAddress> expected = new HashSet<>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
+        Set<InetAddressAndPort> expected = new HashSet<>();
+        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddressAndPort()))
         {
             expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
         }
-        expected.remove(FBUtilities.getBroadcastAddress());
+        expected.remove(FBUtilities.getBroadcastAddressAndPort());
         // remove remote endpoints
         TokenMetadata.Topology topology = tmd.cloneOnlyTokenMap().getTopology();
-        HashSet<InetAddress> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
+        HashSet<InetAddressAndPort> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
         expected = Sets.intersection(expected, localEndpoints);
 
         Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
-        Set<InetAddress> neighbors = new HashSet<>();
+        Set<InetAddressAndPort> neighbors = new HashSet<>();
         for (Range<Token> range : ranges)
         {
             neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null));
@@ -199,14 +199,14 @@ public class ActiveRepairServiceTest
         // generate rf*2 nodes, and ensure that only neighbors specified by the hosts are returned
         addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
         AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
-        List<InetAddress> expected = new ArrayList<>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
+        List<InetAddressAndPort> expected = new ArrayList<>();
+        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddressAndPort()))
         {
             expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
         }
 
-        expected.remove(FBUtilities.getBroadcastAddress());
-        Collection<String> hosts = Arrays.asList(FBUtilities.getBroadcastAddress().getCanonicalHostName(),expected.get(0).getCanonicalHostName());
+        expected.remove(FBUtilities.getBroadcastAddressAndPort());
+        Collection<String> hosts = Arrays.asList(FBUtilities.getBroadcastAddressAndPort().toString(),expected.get(0).toString());
         Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
 
         assertEquals(expected.get(0), ActiveRepairService.getNeighbors(KEYSPACE5, ranges,
@@ -245,13 +245,13 @@ public class ActiveRepairServiceTest
 
     }
 
-    Set<InetAddress> addTokens(int max) throws Throwable
+    Set<InetAddressAndPort> addTokens(int max) throws Throwable
     {
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-        Set<InetAddress> endpoints = new HashSet<>();
+        Set<InetAddressAndPort> endpoints = new HashSet<>();
         for (int i = 1; i <= max; i++)
         {
-            InetAddress endpoint = InetAddress.getByName("127.0.0." + i);
+            InetAddressAndPort endpoint = InetAddressAndPort.getByName("127.0.0." + i);
             tmd.updateNormalToken(tmd.partitioner.getRandomToken(), endpoint);
             endpoints.add(endpoint);
         }
@@ -264,14 +264,14 @@ public class ActiveRepairServiceTest
         ColumnFamilyStore store = prepareColumnFamilyStore();
         UUID prsId = UUID.randomUUID();
         Set<SSTableReader> original = Sets.newHashSet(store.select(View.select(SSTableSet.CANONICAL, (s) -> !s.isRepaired())).sstables);
-        ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store),
+        ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddressAndPort(), Collections.singletonList(store),
                                                                  Collections.singleton(new Range<>(store.getPartitioner().getMinimumToken(),
                                                                                                    store.getPartitioner().getMinimumToken())),
                                                                  true, System.currentTimeMillis(), true, PreviewKind.NONE);
         ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.id, prsId);
 
         UUID prsId2 = UUID.randomUUID();
-        ActiveRepairService.instance.registerParentRepairSession(prsId2, FBUtilities.getBroadcastAddress(),
+        ActiveRepairService.instance.registerParentRepairSession(prsId2, FBUtilities.getBroadcastAddressAndPort(),
                                                                  Collections.singletonList(store),
                                                                  Collections.singleton(new Range<>(store.getPartitioner().getMinimumToken(),
                                                                                                    store.getPartitioner().getMinimumToken())),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/DataResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/DataResolverTest.java b/test/unit/org/apache/cassandra/service/DataResolverTest.java
index ec25fef..f406582 100644
--- a/test/unit/org/apache/cassandra/service/DataResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/DataResolverTest.java
@@ -18,7 +18,6 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -29,6 +28,7 @@ import org.junit.*;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -150,11 +150,11 @@ public class DataResolverTest
     public void testResolveNewerSingleRow() throws UnknownHostException
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
                                                                                                        .add("c1", "v1")
                                                                                                        .buildUpdate())));
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
                                                                                                        .add("c1", "v2")
                                                                                                        .buildUpdate())));
@@ -182,12 +182,12 @@ public class DataResolverTest
     public void testResolveDisjointSingleRow()
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
                                                                                                        .add("c1", "v1")
                                                                                                        .buildUpdate())));
 
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
                                                                                                        .add("c2", "v2")
                                                                                                        .buildUpdate())));
@@ -220,11 +220,11 @@ public class DataResolverTest
     {
 
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
                                                                                                        .add("c1", "v1")
                                                                                                        .buildUpdate())));
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("2")
                                                                                                        .add("c2", "v2")
                                                                                                        .buildUpdate())));
@@ -274,25 +274,25 @@ public class DataResolverTest
                                                                                   .addRangeTombstone(tombstone2)
                                                                                   .buildUpdate();
 
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         UnfilteredPartitionIterator iter1 = iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).addRangeTombstone(tombstone1)
                                                                                   .addRangeTombstone(tombstone2)
                                                                                   .buildUpdate());
         resolver.preprocess(readResponseMessage(peer1, iter1));
         // not covered by any range tombstone
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         UnfilteredPartitionIterator iter2 = iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("0")
                                                                                   .add("c1", "v0")
                                                                                   .buildUpdate());
         resolver.preprocess(readResponseMessage(peer2, iter2));
         // covered by a range tombstone
-        InetAddress peer3 = peer();
+        InetAddressAndPort peer3 = peer();
         UnfilteredPartitionIterator iter3 = iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("10")
                                                                                   .add("c2", "v1")
                                                                                   .buildUpdate());
         resolver.preprocess(readResponseMessage(peer3, iter3));
         // range covered by rt, but newer
-        InetAddress peer4 = peer();
+        InetAddressAndPort peer4 = peer();
         UnfilteredPartitionIterator iter4 = iter(new RowUpdateBuilder(cfm, nowInSec, 2L, dk).clustering("3")
                                                                                   .add("one", "A")
                                                                                   .buildUpdate());
@@ -348,11 +348,11 @@ public class DataResolverTest
     public void testResolveWithOneEmpty()
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
                                                                                                        .add("c2", "v2")
                                                                                                        .buildUpdate())));
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, EmptyIterators.unfilteredPartition(cfm)));
 
         try(PartitionIterator data = resolver.resolve())
@@ -395,11 +395,11 @@ public class DataResolverTest
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
         // one response with columns timestamped before a delete in another response
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
                                                                                                        .add("one", "A")
                                                                                                        .buildUpdate())));
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, fullPartitionDelete(cfm, dk, 1, nowInSec)));
 
         try (PartitionIterator data = resolver.resolve())
@@ -421,20 +421,20 @@ public class DataResolverTest
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 4, System.nanoTime());
         // deletes and columns with interleaved timestamp, with out of order return sequence
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, fullPartitionDelete(cfm, dk, 0, nowInSec)));
         // these columns created after the previous deletion
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
                                                                                                        .add("one", "A")
                                                                                                        .add("two", "A")
                                                                                                        .buildUpdate())));
         //this column created after the next delete
-        InetAddress peer3 = peer();
+        InetAddressAndPort peer3 = peer();
         resolver.preprocess(readResponseMessage(peer3, iter(new RowUpdateBuilder(cfm, nowInSec, 3L, dk).clustering("1")
                                                                                                        .add("two", "B")
                                                                                                        .buildUpdate())));
-        InetAddress peer4 = peer();
+        InetAddressAndPort peer4 = peer();
         resolver.preprocess(readResponseMessage(peer4, fullPartitionDelete(cfm, dk, 2, nowInSec)));
 
         try(PartitionIterator data = resolver.resolve())
@@ -505,8 +505,8 @@ public class DataResolverTest
     private void resolveRangeTombstonesOnBoundary(long timestamp1, long timestamp2)
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
-        InetAddress peer1 = peer();
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer1 = peer();
+        InetAddressAndPort peer2 = peer();
 
         // 1st "stream"
         RangeTombstone one_two    = tombstone("1", true , "2", false, timestamp1, nowInSec);
@@ -579,8 +579,8 @@ public class DataResolverTest
     private void testRepairRangeTombstoneBoundary(int timestamp1, int timestamp2, int timestamp3) throws UnknownHostException
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
-        InetAddress peer1 = peer();
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer1 = peer();
+        InetAddressAndPort peer2 = peer();
 
         // 1st "stream"
         RangeTombstone one_nine = tombstone("0", true , "9", true, timestamp1, nowInSec);
@@ -631,8 +631,8 @@ public class DataResolverTest
     public void testRepairRangeTombstoneWithPartitionDeletion()
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
-        InetAddress peer1 = peer();
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer1 = peer();
+        InetAddressAndPort peer2 = peer();
 
         // 1st "stream": just a partition deletion
         UnfilteredPartitionIterator iter1 = iter(PartitionUpdate.fullPartitionDelete(cfm, dk, 10, nowInSec));
@@ -670,8 +670,8 @@ public class DataResolverTest
     public void testRepairRangeTombstoneWithPartitionDeletion2()
     {
         DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime());
-        InetAddress peer1 = peer();
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer1 = peer();
+        InetAddressAndPort peer2 = peer();
 
         // 1st "stream": a partition deletion and a range tombstone
         RangeTombstone rt1 = tombstone("0", true , "9", true, 11, nowInSec);
@@ -761,7 +761,7 @@ public class DataResolverTest
         builder.addComplexDeletion(m, new DeletionTime(ts[0] - 1, nowInSec));
         builder.addCell(mapCell(0, 0, ts[0]));
 
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
 
         builder.newRow(Clustering.EMPTY);
@@ -770,7 +770,7 @@ public class DataResolverTest
         Cell expectedCell = mapCell(1, 1, ts[1]);
         builder.addCell(expectedCell);
 
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
 
         try(PartitionIterator data = resolver.resolve())
@@ -814,14 +814,14 @@ public class DataResolverTest
         builder.addComplexDeletion(m, new DeletionTime(ts[0] - 1, nowInSec));
         builder.addCell(mapCell(0, 0, ts[0]));
 
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
 
         builder.newRow(Clustering.EMPTY);
         DeletionTime expectedCmplxDelete = new DeletionTime(ts[1] - 1, nowInSec);
         builder.addComplexDeletion(m, expectedCmplxDelete);
 
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
 
         try(PartitionIterator data = resolver.resolve())
@@ -862,10 +862,10 @@ public class DataResolverTest
         builder.addCell(expectedCell);
 
         // empty map column
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
 
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, iter(PartitionUpdate.emptyUpdate(cfm2, dk))));
 
         try(PartitionIterator data = resolver.resolve())
@@ -908,7 +908,7 @@ public class DataResolverTest
         builder.newRow(Clustering.EMPTY);
         builder.addComplexDeletion(m, new DeletionTime(ts[0] - 1, nowInSec));
 
-        InetAddress peer1 = peer();
+        InetAddressAndPort peer1 = peer();
         resolver.preprocess(readResponseMessage(peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
 
         // newer, overwritten map column
@@ -918,7 +918,7 @@ public class DataResolverTest
         Cell expectedCell = mapCell(1, 1, ts[1]);
         builder.addCell(expectedCell);
 
-        InetAddress peer2 = peer();
+        InetAddressAndPort peer2 = peer();
         resolver.preprocess(readResponseMessage(peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
 
         try(PartitionIterator data = resolver.resolve())
@@ -945,11 +945,11 @@ public class DataResolverTest
         Assert.assertNull(messageRecorder.sent.get(peer2));
     }
 
-    private InetAddress peer()
+    private InetAddressAndPort peer()
     {
         try
         {
-            return InetAddress.getByAddress(new byte[]{ 127, 0, 0, (byte) addressSuffix++ });
+            return InetAddressAndPort.getByAddress(new byte[]{ 127, 0, 0, (byte) addressSuffix++ });
         }
         catch (UnknownHostException e)
         {
@@ -957,7 +957,7 @@ public class DataResolverTest
         }
     }
 
-    private MessageOut<Mutation> getSentMessage(InetAddress target)
+    private MessageOut<Mutation> getSentMessage(InetAddressAndPort target)
     {
         MessageOut<Mutation> message = messageRecorder.sent.get(target);
         assertNotNull(String.format("No repair message was sent to %s", target), message);
@@ -1018,12 +1018,12 @@ public class DataResolverTest
     }
 
 
-    public MessageIn<ReadResponse> readResponseMessage(InetAddress from, UnfilteredPartitionIterator partitionIterator)
+    public MessageIn<ReadResponse> readResponseMessage(InetAddressAndPort from, UnfilteredPartitionIterator partitionIterator)
     {
         return readResponseMessage(from, partitionIterator, command);
 
     }
-    public MessageIn<ReadResponse> readResponseMessage(InetAddress from, UnfilteredPartitionIterator partitionIterator, ReadCommand cmd)
+    public MessageIn<ReadResponse> readResponseMessage(InetAddressAndPort from, UnfilteredPartitionIterator partitionIterator, ReadCommand cmd)
     {
         return MessageIn.create(from,
                                 ReadResponse.createRemoteDataResponse(partitionIterator, cmd),
@@ -1080,8 +1080,8 @@ public class DataResolverTest
 
     private static class MessageRecorder implements IMessageSink
     {
-        Map<InetAddress, MessageOut> sent = new HashMap<>();
-        public boolean allowOutgoingMessage(MessageOut message, int id, InetAddress to)
+        Map<InetAddressAndPort, MessageOut> sent = new HashMap<>();
+        public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to)
         {
             sent.put(to, message);
             return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
index aaa2594..8ddc4f0 100644
--- a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
+++ b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
@@ -19,7 +19,6 @@
 
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 
@@ -34,6 +33,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.Util.PartitionerSwitcher;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.IPartitioner;
@@ -92,17 +92,17 @@ public class LeaveAndBootstrapTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);
 
-        Map<Token, List<InetAddress>> expectedEndpoints = new HashMap<Token, List<InetAddress>>();
+        Map<Token, List<InetAddressAndPort>> expectedEndpoints = new HashMap<Token, List<InetAddressAndPort>>();
         for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
         {
             for (Token token : keyTokens)
             {
-                List<InetAddress> endpoints = new ArrayList<InetAddress>();
+                List<InetAddressAndPort> endpoints = new ArrayList<>();
                 Iterator<Token> tokenIter = TokenMetadata.ringIterator(tmd.sortedTokens(), token, false);
                 while (tokenIter.hasNext())
                 {
@@ -114,6 +114,9 @@ public class LeaveAndBootstrapTest
 
         // Third node leaves
         ss.onChange(hosts.get(LEAVING_NODE),
+                    ApplicationState.STATUS_WITH_PORT,
+                    valueFactory.leaving(Collections.singleton(endpointTokens.get(LEAVING_NODE))));
+        ss.onChange(hosts.get(LEAVING_NODE),
                 ApplicationState.STATUS,
                 valueFactory.leaving(Collections.singleton(endpointTokens.get(LEAVING_NODE))));
         assertTrue(tmd.isLeaving(hosts.get(LEAVING_NODE)));
@@ -129,8 +132,8 @@ public class LeaveAndBootstrapTest
             {
                 int replicationFactor = strategy.getReplicationFactor();
 
-                HashSet<InetAddress> actual = new HashSet<InetAddress>(tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
-                HashSet<InetAddress> expected = new HashSet<InetAddress>();
+                HashSet<InetAddressAndPort> actual = new HashSet<>(tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
+                HashSet<InetAddressAndPort> expected = new HashSet<>();
 
                 for (int i = 0; i < replicationFactor; i++)
                 {
@@ -163,34 +166,39 @@ public class LeaveAndBootstrapTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         // create a ring or 10 nodes
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);
 
         // nodes 6, 8 and 9 leave
-        final int[] LEAVING = new int[] {6, 8, 9};
+        final int[] LEAVING = new int[]{ 6, 8, 9 };
         for (int leaving : LEAVING)
+        {
+            ss.onChange(hosts.get(leaving),
+                        ApplicationState.STATUS_WITH_PORT,
+                        valueFactory.leaving(Collections.singleton(endpointTokens.get(leaving))));
             ss.onChange(hosts.get(leaving),
                         ApplicationState.STATUS,
                         valueFactory.leaving(Collections.singleton(endpointTokens.get(leaving))));
+        }
 
         // boot two new nodes with keyTokens.get(5) and keyTokens.get(7)
-        InetAddress boot1 = InetAddress.getByName("127.0.1.1");
+        InetAddressAndPort boot1 = InetAddressAndPort.getByName("127.0.1.1");
         Gossiper.instance.initializeNodeUnsafe(boot1, UUID.randomUUID(), 1);
         Gossiper.instance.injectApplicationState(boot1, ApplicationState.TOKENS, valueFactory.tokens(Collections.singleton(keyTokens.get(5))));
         ss.onChange(boot1,
                     ApplicationState.STATUS,
                     valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(5))));
-        InetAddress boot2 = InetAddress.getByName("127.0.1.2");
+        InetAddressAndPort boot2 = InetAddressAndPort.getByName("127.0.1.2");
         Gossiper.instance.initializeNodeUnsafe(boot2, UUID.randomUUID(), 1);
         Gossiper.instance.injectApplicationState(boot2, ApplicationState.TOKENS, valueFactory.tokens(Collections.singleton(keyTokens.get(7))));
         ss.onChange(boot2,
                     ApplicationState.STATUS,
                     valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(7))));
 
-        Collection<InetAddress> endpoints = null;
+        Collection<InetAddressAndPort> endpoints = null;
 
         /* don't require test update every time a new keyspace is added to test/conf/cassandra.yaml */
         Map<String, AbstractReplicationStrategy> keyspaceStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
@@ -200,8 +208,8 @@ public class LeaveAndBootstrapTest
         }
 
         // pre-calculate the results.
-        Map<String, Multimap<Token, InetAddress>> expectedEndpoints = new HashMap<String, Multimap<Token, InetAddress>>();
-        expectedEndpoints.put(KEYSPACE1, HashMultimap.<Token, InetAddress>create());
+        Map<String, Multimap<Token, InetAddressAndPort>> expectedEndpoints = new HashMap<String, Multimap<Token, InetAddressAndPort>>();
+        expectedEndpoints.put(KEYSPACE1, HashMultimap.<Token, InetAddressAndPort>create());
         expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
         expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
         expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
@@ -212,7 +220,7 @@ public class LeaveAndBootstrapTest
         expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
         expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
         expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
-        expectedEndpoints.put(KEYSPACE2, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.put(KEYSPACE2, HashMultimap.<Token, InetAddressAndPort>create());
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
@@ -223,7 +231,7 @@ public class LeaveAndBootstrapTest
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
-        expectedEndpoints.put(KEYSPACE3, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.put(KEYSPACE3, HashMultimap.<Token, InetAddressAndPort>create());
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
@@ -234,7 +242,7 @@ public class LeaveAndBootstrapTest
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2", "127.0.0.4", "127.0.0.5"));
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.put(KEYSPACE4, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.put(KEYSPACE4, HashMultimap.<Token, InetAddressAndPort>create());
         expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
         expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
         expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
@@ -463,7 +471,7 @@ public class LeaveAndBootstrapTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         // create a ring or 5 nodes
@@ -540,7 +548,7 @@ public class LeaveAndBootstrapTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         // create a ring or 5 nodes
@@ -582,7 +590,7 @@ public class LeaveAndBootstrapTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         // create a ring or 5 nodes
@@ -632,7 +640,7 @@ public class LeaveAndBootstrapTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         // create a ring of 6 nodes
@@ -673,10 +681,10 @@ public class LeaveAndBootstrapTest
 
         // create a ring of 2 nodes
         ArrayList<Token> endpointTokens = new ArrayList<>();
-        List<InetAddress> hosts = new ArrayList<>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         Util.createInitialRing(ss, partitioner, endpointTokens, new ArrayList<Token>(), hosts, new ArrayList<UUID>(), 2);
 
-        InetAddress toRemove = hosts.get(1);
+        InetAddressAndPort toRemove = hosts.get(1);
         SystemKeyspace.updatePeerInfo(toRemove, "data_center", "dc42");
         SystemKeyspace.updatePeerInfo(toRemove, "rack", "rack42");
         assertEquals("rack42", SystemKeyspace.loadDcRackInfo().get(toRemove).get("rack"));
@@ -698,18 +706,19 @@ public class LeaveAndBootstrapTest
         // create a ring of 1 node
         StorageService ss = StorageService.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
-        Util.createInitialRing(ss, partitioner, new ArrayList<Token>(), new ArrayList<Token>(), new ArrayList<InetAddress>(), new ArrayList<UUID>(), 1);
+        Util.createInitialRing(ss, partitioner, new ArrayList<Token>(), new ArrayList<Token>(), new ArrayList<InetAddressAndPort>(), new ArrayList<UUID>(), 1);
 
         // make a REMOVING state change on a non-member endpoint; without the CASSANDRA-6564 fix, this
         // would result in an ArrayIndexOutOfBoundsException
-        ss.onChange(InetAddress.getByName("192.168.1.42"), ApplicationState.STATUS, valueFactory.removingNonlocal(UUID.randomUUID()));
+        ss.onChange(InetAddressAndPort.getByName("192.168.1.42"), ApplicationState.STATUS_WITH_PORT, valueFactory.removingNonlocal(UUID.randomUUID()));
+        ss.onChange(InetAddressAndPort.getByName("192.168.1.42"), ApplicationState.STATUS, valueFactory.removingNonlocal(UUID.randomUUID()));
     }
 
-    private static Collection<InetAddress> makeAddrs(String... hosts) throws UnknownHostException
+    private static Collection<InetAddressAndPort> makeAddrs(String... hosts) throws UnknownHostException
     {
-        ArrayList<InetAddress> addrs = new ArrayList<InetAddress>(hosts.length);
+        ArrayList<InetAddressAndPort> addrs = new ArrayList<>(hosts.length);
         for (String host : hosts)
-            addrs.add(InetAddress.getByName(host));
+            addrs.add(InetAddressAndPort.getByName(host));
         return addrs;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/MoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java b/test/unit/org/apache/cassandra/service/MoveTest.java
index df4e294..7321fba 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -19,7 +19,6 @@
 
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 
@@ -30,6 +29,7 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -115,7 +115,7 @@ public class MoveTest
             //Odd IPs are in DC1 and Even are in DC2. Endpoints upto .14 will have unique racks and
             // then will be same for a set of three.
             @Override
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 int ipLastPart = getIPLastPart(endpoint);
                 if (ipLastPart <= 14)
@@ -125,7 +125,7 @@ public class MoveTest
             }
 
             @Override
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
                 if (getIPLastPart(endpoint) % 2 == 0)
                     return "DC2";
@@ -133,19 +133,19 @@ public class MoveTest
                     return "DC1";
             }
 
-            private int getIPLastPart(InetAddress endpoint)
+            private int getIPLastPart(InetAddressAndPort endpoint)
             {
                 String str = endpoint.toString();
                 int index = str.lastIndexOf(".");
-                return Integer.parseInt(str.substring(index + 1).trim());
+                return Integer.parseInt(str.substring(index + 1).trim().split(":")[0]);
             }
         });
 
         final TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         tmd.clearUnsafe();
-        tmd.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.0.0.1"));
-        tmd.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.0.0.2"));
+        tmd.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.0.0.1"));
+        tmd.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.0.0.2"));
 
         KeyspaceMetadata keyspace =  KeyspaceMetadata.create(keyspaceName,
                                                              KeyspaceParams.nts(configOptions(replicas)),
@@ -181,7 +181,7 @@ public class MoveTest
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
         ArrayList<Token> endpointTokens = new ArrayList<>();
         ArrayList<Token> keyTokens = new ArrayList<>();
-        List<InetAddress> hosts = new ArrayList<>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<>();
 
         for(int i=0; i < RING_SIZE/2; i++)
@@ -221,7 +221,7 @@ public class MoveTest
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
         ArrayList<Token> endpointTokens = new ArrayList<>();
         ArrayList<Token> keyTokens = new ArrayList<>();
-        List<InetAddress> hosts = new ArrayList<>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<>();
 
         for(int i=0; i < RING_SIZE/2; i++)
@@ -312,7 +312,7 @@ public class MoveTest
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
         ArrayList<Token> endpointTokens = new ArrayList<>();
         ArrayList<Token> keyTokens = new ArrayList<>();
-        List<InetAddress> hosts = new ArrayList<>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<>();
 
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);
@@ -376,7 +376,7 @@ public class MoveTest
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
         ArrayList<Token> endpointTokens = new ArrayList<>();
         ArrayList<Token> keyTokens = new ArrayList<>();
-        List<InetAddress> hosts = new ArrayList<>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<>();
 
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);
@@ -478,38 +478,38 @@ public class MoveTest
         finishMove(hosts.get(MOVING_NODE), 30, tmd);
     }
 
-    private void moveHost(InetAddress host, int token, TokenMetadata tmd, VersionedValue.VersionedValueFactory valueFactory )
+    private void moveHost(InetAddressAndPort host, int token, TokenMetadata tmd, VersionedValue.VersionedValueFactory valueFactory )
     {
         StorageService.instance.onChange(host, ApplicationState.STATUS, valueFactory.moving(new BigIntegerToken(String.valueOf(token))));
         PendingRangeCalculatorService.instance.blockUntilFinished();
         assertTrue(tmd.isMoving(host));
     }
 
-    private void finishMove(InetAddress host, int token, TokenMetadata tmd)
+    private void finishMove(InetAddressAndPort host, int token, TokenMetadata tmd)
     {
         tmd.removeFromMoving(host);
         assertTrue(!tmd.isMoving(host));
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(token)), host);
     }
 
-    private Map.Entry<Range<Token>, Collection<InetAddress>> generatePendingMapEntry(int start, int end, String... endpoints) throws UnknownHostException
+    private Map.Entry<Range<Token>, Collection<InetAddressAndPort>> generatePendingMapEntry(int start, int end, String... endpoints) throws UnknownHostException
     {
-        Map<Range<Token>, Collection<InetAddress>> pendingRanges = new HashMap<>();
+        Map<Range<Token>, Collection<InetAddressAndPort>> pendingRanges = new HashMap<>();
         pendingRanges.put(generateRange(start, end), makeAddrs(endpoints));
         return pendingRanges.entrySet().iterator().next();
     }
 
-    private Map<Range<Token>, Collection<InetAddress>> generatePendingRanges(Map.Entry<Range<Token>, Collection<InetAddress>>... entries)
+    private Map<Range<Token>, Collection<InetAddressAndPort>> generatePendingRanges(Map.Entry<Range<Token>, Collection<InetAddressAndPort>>... entries)
     {
-        Map<Range<Token>, Collection<InetAddress>> pendingRanges = new HashMap<>();
-        for(Map.Entry<Range<Token>, Collection<InetAddress>> entry : entries)
+        Map<Range<Token>, Collection<InetAddressAndPort>> pendingRanges = new HashMap<>();
+        for(Map.Entry<Range<Token>, Collection<InetAddressAndPort>> entry : entries)
         {
             pendingRanges.put(entry.getKey(), entry.getValue());
         }
         return pendingRanges;
     }
 
-    private void assertPendingRanges(TokenMetadata tmd, Map<Range<Token>,  Collection<InetAddress>> pendingRanges, String keyspaceName) throws ConfigurationException
+    private void assertPendingRanges(TokenMetadata tmd, Map<Range<Token>,  Collection<InetAddressAndPort>> pendingRanges, String keyspaceName) throws ConfigurationException
     {
         boolean keyspaceFound = false;
         for (String nonSystemKeyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
@@ -523,13 +523,13 @@ public class MoveTest
         assert keyspaceFound;
     }
 
-    private void assertMaps(Map<Range<Token>, Collection<InetAddress>> expected, PendingRangeMaps actual)
+    private void assertMaps(Map<Range<Token>, Collection<InetAddressAndPort>> expected, PendingRangeMaps actual)
     {
         int sizeOfActual = 0;
-        Iterator<Map.Entry<Range<Token>, List<InetAddress>>> iterator = actual.iterator();
+        Iterator<Map.Entry<Range<Token>, List<InetAddressAndPort>>> iterator = actual.iterator();
         while(iterator.hasNext())
         {
-            Map.Entry<Range<Token>, List<InetAddress>> actualEntry = iterator.next();
+            Map.Entry<Range<Token>, List<InetAddressAndPort>> actualEntry = iterator.next();
             assertNotNull(expected.get(actualEntry.getKey()));
             assertEquals(new HashSet<>(expected.get(actualEntry.getKey())), new HashSet<>(actualEntry.getValue()));
             sizeOfActual++;
@@ -554,15 +554,15 @@ public class MoveTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);
 
-        Map<Token, List<InetAddress>> expectedEndpoints = new HashMap<Token, List<InetAddress>>();
+        Map<Token, List<InetAddressAndPort>> expectedEndpoints = new HashMap<>();
         for (Token token : keyTokens)
         {
-            List<InetAddress> endpoints = new ArrayList<InetAddress>();
+            List<InetAddressAndPort> endpoints = new ArrayList<>();
             Iterator<Token> tokenIter = TokenMetadata.ringIterator(tmd.sortedTokens(), token, false);
             while (tokenIter.hasNext())
             {
@@ -591,8 +591,8 @@ public class MoveTest
             {
                 int replicationFactor = strategy.getReplicationFactor();
 
-                HashSet<InetAddress> actual = new HashSet<InetAddress>(tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
-                HashSet<InetAddress> expected = new HashSet<InetAddress>();
+                HashSet<InetAddressAndPort> actual = new HashSet<>(tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
+                HashSet<InetAddressAndPort> expected = new HashSet<>();
 
                 for (int i = 0; i < replicationFactor; i++)
                 {
@@ -628,7 +628,7 @@ public class MoveTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         // create a ring or 10 nodes
@@ -648,13 +648,13 @@ public class MoveTest
             newTokens.put(movingIndex, newToken);
         }
 
-        Collection<InetAddress> endpoints;
+        Collection<InetAddressAndPort> endpoints;
 
         tmd = tmd.cloneAfterAllSettled();
         ss.setTokenMetadataUnsafe(tmd);
 
         // boot two new nodes with keyTokens.get(5) and keyTokens.get(7)
-        InetAddress boot1 = InetAddress.getByName("127.0.1.1");
+        InetAddressAndPort boot1 = InetAddressAndPort.getByName("127.0.1.1");
         Gossiper.instance.initializeNodeUnsafe(boot1, UUID.randomUUID(), 1);
         Gossiper.instance.injectApplicationState(boot1, ApplicationState.TOKENS, valueFactory.tokens(Collections.singleton(keyTokens.get(5))));
         ss.onChange(boot1,
@@ -662,7 +662,7 @@ public class MoveTest
                     valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(5))));
         PendingRangeCalculatorService.instance.blockUntilFinished();
 
-        InetAddress boot2 = InetAddress.getByName("127.0.1.2");
+        InetAddressAndPort boot2 = InetAddressAndPort.getByName("127.0.1.2");
         Gossiper.instance.initializeNodeUnsafe(boot2, UUID.randomUUID(), 1);
         Gossiper.instance.injectApplicationState(boot2, ApplicationState.TOKENS, valueFactory.tokens(Collections.singleton(keyTokens.get(7))));
         ss.onChange(boot2,
@@ -693,35 +693,35 @@ public class MoveTest
         *  }
         */
 
-        Multimap<InetAddress, Range<Token>> keyspace1ranges = keyspaceStrategyMap.get(Simple_RF1_KeyspaceName).getAddressRanges();
-        Collection<Range<Token>> ranges1 = keyspace1ranges.get(InetAddress.getByName("127.0.0.1"));
+        Multimap<InetAddressAndPort, Range<Token>> keyspace1ranges = keyspaceStrategyMap.get(Simple_RF1_KeyspaceName).getAddressRanges();
+        Collection<Range<Token>> ranges1 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.1"));
         assertEquals(1, collectionSize(ranges1));
         assertEquals(generateRange(97, 0), ranges1.iterator().next());
-        Collection<Range<Token>> ranges2 = keyspace1ranges.get(InetAddress.getByName("127.0.0.2"));
+        Collection<Range<Token>> ranges2 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.2"));
         assertEquals(1, collectionSize(ranges2));
         assertEquals(generateRange(0, 10), ranges2.iterator().next());
-        Collection<Range<Token>> ranges3 = keyspace1ranges.get(InetAddress.getByName("127.0.0.3"));
+        Collection<Range<Token>> ranges3 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.3"));
         assertEquals(1, collectionSize(ranges3));
         assertEquals(generateRange(10, 20), ranges3.iterator().next());
-        Collection<Range<Token>> ranges4 = keyspace1ranges.get(InetAddress.getByName("127.0.0.4"));
+        Collection<Range<Token>> ranges4 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.4"));
         assertEquals(1, collectionSize(ranges4));
         assertEquals(generateRange(20, 30), ranges4.iterator().next());
-        Collection<Range<Token>> ranges5 = keyspace1ranges.get(InetAddress.getByName("127.0.0.5"));
+        Collection<Range<Token>> ranges5 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.5"));
         assertEquals(1, collectionSize(ranges5));
         assertEquals(generateRange(30, 40), ranges5.iterator().next());
-        Collection<Range<Token>> ranges6 = keyspace1ranges.get(InetAddress.getByName("127.0.0.6"));
+        Collection<Range<Token>> ranges6 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.6"));
         assertEquals(1, collectionSize(ranges6));
         assertEquals(generateRange(40, 50), ranges6.iterator().next());
-        Collection<Range<Token>> ranges7 = keyspace1ranges.get(InetAddress.getByName("127.0.0.7"));
+        Collection<Range<Token>> ranges7 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.7"));
         assertEquals(1, collectionSize(ranges7));
         assertEquals(generateRange(50, 67), ranges7.iterator().next());
-        Collection<Range<Token>> ranges8 = keyspace1ranges.get(InetAddress.getByName("127.0.0.8"));
+        Collection<Range<Token>> ranges8 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.8"));
         assertEquals(1, collectionSize(ranges8));
         assertEquals(generateRange(67, 70), ranges8.iterator().next());
-        Collection<Range<Token>> ranges9 = keyspace1ranges.get(InetAddress.getByName("127.0.0.9"));
+        Collection<Range<Token>> ranges9 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.9"));
         assertEquals(1, collectionSize(ranges9));
         assertEquals(generateRange(70, 87), ranges9.iterator().next());
-        Collection<Range<Token>> ranges10 = keyspace1ranges.get(InetAddress.getByName("127.0.0.10"));
+        Collection<Range<Token>> ranges10 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.10"));
         assertEquals(1, collectionSize(ranges10));
         assertEquals(generateRange(87, 97), ranges10.iterator().next());
 
@@ -742,35 +742,35 @@ public class MoveTest
         * }
         */
 
-        Multimap<InetAddress, Range<Token>> keyspace3ranges = keyspaceStrategyMap.get(KEYSPACE3).getAddressRanges();
-        ranges1 = keyspace3ranges.get(InetAddress.getByName("127.0.0.1"));
+        Multimap<InetAddressAndPort, Range<Token>> keyspace3ranges = keyspaceStrategyMap.get(KEYSPACE3).getAddressRanges();
+        ranges1 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 5);
         assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 50, 67, 87, 97, 67, 70)));
-        ranges2 = keyspace3ranges.get(InetAddress.getByName("127.0.0.2"));
+        ranges2 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.2"));
         assertEquals(collectionSize(ranges2), 5);
         assertTrue(ranges2.equals(generateRanges(97, 0, 70, 87, 87, 97, 0, 10, 67, 70)));
-        ranges3 = keyspace3ranges.get(InetAddress.getByName("127.0.0.3"));
+        ranges3 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.3"));
         assertEquals(collectionSize(ranges3), 5);
         assertTrue(ranges3.equals(generateRanges(97, 0, 70, 87, 87, 97, 0, 10, 10, 20)));
-        ranges4 = keyspace3ranges.get(InetAddress.getByName("127.0.0.4"));
+        ranges4 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.4"));
         assertEquals(collectionSize(ranges4), 5);
         assertTrue(ranges4.equals(generateRanges(97, 0, 20, 30, 87, 97, 0, 10, 10, 20)));
-        ranges5 = keyspace3ranges.get(InetAddress.getByName("127.0.0.5"));
+        ranges5 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.5"));
         assertEquals(collectionSize(ranges5), 5);
         assertTrue(ranges5.equals(generateRanges(97, 0, 30, 40, 20, 30, 0, 10, 10, 20)));
-        ranges6 = keyspace3ranges.get(InetAddress.getByName("127.0.0.6"));
+        ranges6 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.6"));
         assertEquals(collectionSize(ranges6), 5);
         assertTrue(ranges6.equals(generateRanges(40, 50, 30, 40, 20, 30, 0, 10, 10, 20)));
-        ranges7 = keyspace3ranges.get(InetAddress.getByName("127.0.0.7"));
+        ranges7 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.7"));
         assertEquals(collectionSize(ranges7), 5);
         assertTrue(ranges7.equals(generateRanges(40, 50, 30, 40, 50, 67, 20, 30, 10, 20)));
-        ranges8 = keyspace3ranges.get(InetAddress.getByName("127.0.0.8"));
+        ranges8 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.8"));
         assertEquals(collectionSize(ranges8), 5);
         assertTrue(ranges8.equals(generateRanges(40, 50, 30, 40, 50, 67, 20, 30, 67, 70)));
-        ranges9 = keyspace3ranges.get(InetAddress.getByName("127.0.0.9"));
+        ranges9 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.9"));
         assertEquals(collectionSize(ranges9), 5);
         assertTrue(ranges9.equals(generateRanges(40, 50, 70, 87, 30, 40, 50, 67, 67, 70)));
-        ranges10 = keyspace3ranges.get(InetAddress.getByName("127.0.0.10"));
+        ranges10 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.10"));
         assertEquals(collectionSize(ranges10), 5);
         assertTrue(ranges10.equals(generateRanges(40, 50, 70, 87, 50, 67, 87, 97, 67, 70)));
 
@@ -790,41 +790,41 @@ public class MoveTest
          *      /127.0.0.10=[(70,87], (87,97], (67,70]]
          *  }
          */
-        Multimap<InetAddress, Range<Token>> keyspace4ranges = keyspaceStrategyMap.get(Simple_RF3_KeyspaceName).getAddressRanges();
-        ranges1 = keyspace4ranges.get(InetAddress.getByName("127.0.0.1"));
+        Multimap<InetAddressAndPort, Range<Token>> keyspace4ranges = keyspaceStrategyMap.get(Simple_RF3_KeyspaceName).getAddressRanges();
+        ranges1 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 3);
         assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 87, 97)));
-        ranges2 = keyspace4ranges.get(InetAddress.getByName("127.0.0.2"));
+        ranges2 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.2"));
         assertEquals(collectionSize(ranges2), 3);
         assertTrue(ranges2.equals(generateRanges(97, 0, 87, 97, 0, 10)));
-        ranges3 = keyspace4ranges.get(InetAddress.getByName("127.0.0.3"));
+        ranges3 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.3"));
         assertEquals(collectionSize(ranges3), 3);
         assertTrue(ranges3.equals(generateRanges(97, 0, 0, 10, 10, 20)));
-        ranges4 = keyspace4ranges.get(InetAddress.getByName("127.0.0.4"));
+        ranges4 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.4"));
         assertEquals(collectionSize(ranges4), 3);
         assertTrue(ranges4.equals(generateRanges(20, 30, 0, 10, 10, 20)));
-        ranges5 = keyspace4ranges.get(InetAddress.getByName("127.0.0.5"));
+        ranges5 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.5"));
         assertEquals(collectionSize(ranges5), 3);
         assertTrue(ranges5.equals(generateRanges(30, 40, 20, 30, 10, 20)));
-        ranges6 = keyspace4ranges.get(InetAddress.getByName("127.0.0.6"));
+        ranges6 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.6"));
         assertEquals(collectionSize(ranges6), 3);
         assertTrue(ranges6.equals(generateRanges(40, 50, 30, 40, 20, 30)));
-        ranges7 = keyspace4ranges.get(InetAddress.getByName("127.0.0.7"));
+        ranges7 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.7"));
         assertEquals(collectionSize(ranges7), 3);
         assertTrue(ranges7.equals(generateRanges(40, 50, 30, 40, 50, 67)));
-        ranges8 = keyspace4ranges.get(InetAddress.getByName("127.0.0.8"));
+        ranges8 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.8"));
         assertEquals(collectionSize(ranges8), 3);
         assertTrue(ranges8.equals(generateRanges(40, 50, 50, 67, 67, 70)));
-        ranges9 = keyspace4ranges.get(InetAddress.getByName("127.0.0.9"));
+        ranges9 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.9"));
         assertEquals(collectionSize(ranges9), 3);
         assertTrue(ranges9.equals(generateRanges(70, 87, 50, 67, 67, 70)));
-        ranges10 = keyspace4ranges.get(InetAddress.getByName("127.0.0.10"));
+        ranges10 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.10"));
         assertEquals(collectionSize(ranges10), 3);
         assertTrue(ranges10.equals(generateRanges(70, 87, 87, 97, 67, 70)));
 
         // pre-calculate the results.
-        Map<String, Multimap<Token, InetAddress>> expectedEndpoints = new HashMap<String, Multimap<Token, InetAddress>>();
-        expectedEndpoints.put(Simple_RF1_KeyspaceName, HashMultimap.<Token, InetAddress>create());
+        Map<String, Multimap<Token, InetAddressAndPort>> expectedEndpoints = new HashMap<>();
+        expectedEndpoints.put(Simple_RF1_KeyspaceName, HashMultimap.<Token, InetAddressAndPort>create());
         expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
         expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
         expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
@@ -835,7 +835,7 @@ public class MoveTest
         expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
         expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
         expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
-        expectedEndpoints.put(KEYSPACE2, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.put(KEYSPACE2, HashMultimap.<Token, InetAddressAndPort>create());
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
@@ -846,7 +846,7 @@ public class MoveTest
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
         expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
-        expectedEndpoints.put(KEYSPACE3, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.put(KEYSPACE3, HashMultimap.<Token, InetAddressAndPort>create());
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
@@ -857,7 +857,7 @@ public class MoveTest
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2"));
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
         expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.put(Simple_RF3_KeyspaceName, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.put(Simple_RF3_KeyspaceName, HashMultimap.<Token, InetAddressAndPort>create());
         expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
         expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
         expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
@@ -970,7 +970,7 @@ public class MoveTest
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
         ArrayList<Token> keyTokens = new ArrayList<Token>();
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<UUID>();
 
         // create a ring or 6 nodes
@@ -1001,11 +1001,11 @@ public class MoveTest
         assertEquals(newToken, tmd.getToken(hosts.get(2)));
     }
 
-    private static Collection<InetAddress> makeAddrs(String... hosts) throws UnknownHostException
+    private static Collection<InetAddressAndPort> makeAddrs(String... hosts) throws UnknownHostException
     {
-        ArrayList<InetAddress> addrs = new ArrayList<InetAddress>(hosts.length);
+        ArrayList<InetAddressAndPort> addrs = new ArrayList<>(hosts.length);
         for (String host : hosts)
-            addrs.add(InetAddress.getByName(host));
+            addrs.add(InetAddressAndPort.getByName(host));
         return addrs;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/ProtocolBetaVersionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ProtocolBetaVersionTest.java b/test/unit/org/apache/cassandra/service/ProtocolBetaVersionTest.java
index 4ade4ad..a7551f4 100644
--- a/test/unit/org/apache/cassandra/service/ProtocolBetaVersionTest.java
+++ b/test/unit/org/apache/cassandra/service/ProtocolBetaVersionTest.java
@@ -110,7 +110,7 @@ public class ProtocolBetaVersionTest extends CQLTester
         }
         catch (Exception e)
         {
-            assertEquals("Beta version of server used (5/v5-beta), but USE_BETA flag is not set",
+            assertEquals("Beta version of server used (6/v6-beta), but USE_BETA flag is not set",
                          e.getMessage());
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/ReadExecutorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ReadExecutorTest.java b/test/unit/org/apache/cassandra/service/ReadExecutorTest.java
index 7630cc6..f21e241 100644
--- a/test/unit/org/apache/cassandra/service/ReadExecutorTest.java
+++ b/test/unit/org/apache/cassandra/service/ReadExecutorTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
@@ -36,6 +35,7 @@ import org.apache.cassandra.db.SinglePartitionReadCommand;
 import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -47,7 +47,7 @@ public class ReadExecutorTest
 {
     static Keyspace ks;
     static ColumnFamilyStore cfs;
-    static List<InetAddress> targets;
+    static List<InetAddressAndPort> targets;
 
     @BeforeClass
     public static void setUpClass() throws Throwable
@@ -56,7 +56,7 @@ public class ReadExecutorTest
         SchemaLoader.createKeyspace("Foo", KeyspaceParams.simple(3), SchemaLoader.standardCFMD("Foo", "Bar"));
         ks = Keyspace.open("Foo");
         cfs = ks.getColumnFamilyStore("Bar");
-        targets = ImmutableList.of(InetAddress.getByName("127.0.0.255"), InetAddress.getByName("127.0.0.254"), InetAddress.getByName("127.0.0.253"));
+        targets = ImmutableList.of(InetAddressAndPort.getByName("127.0.0.255"), InetAddressAndPort.getByName("127.0.0.254"), InetAddressAndPort.getByName("127.0.0.253"));
         cfs.sampleLatencyNanos = 0;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 afc5b25..6714a83 100644
--- a/test/unit/org/apache/cassandra/service/RemoveTest.java
+++ b/test/unit/org/apache/cassandra/service/RemoveTest.java
@@ -20,7 +20,6 @@
 package org.apache.cassandra.service;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -40,6 +39,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.VersionedValue.VersionedValueFactory;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
@@ -61,9 +61,9 @@ public class RemoveTest
     static IPartitioner oldPartitioner;
     ArrayList<Token> endpointTokens = new ArrayList<Token>();
     ArrayList<Token> keyTokens = new ArrayList<Token>();
-    List<InetAddress> hosts = new ArrayList<InetAddress>();
+    List<InetAddressAndPort> hosts = new ArrayList<>();
     List<UUID> hostIds = new ArrayList<UUID>();
-    InetAddress removalhost;
+    InetAddressAndPort removalhost;
     UUID removalId;
 
     @BeforeClass
@@ -120,7 +120,7 @@ public class RemoveTest
         VersionedValueFactory valueFactory = new VersionedValueFactory(DatabaseDescriptor.getPartitioner());
         Collection<Token> tokens = Collections.singleton(DatabaseDescriptor.getPartitioner().getRandomToken());
 
-        InetAddress joininghost = hosts.get(4);
+        InetAddressAndPort joininghost = hosts.get(4);
         UUID joiningId = hostIds.get(4);
 
         hosts.remove(joininghost);
@@ -159,10 +159,10 @@ public class RemoveTest
         assertTrue(tmd.isLeaving(removalhost));
         assertEquals(1, tmd.getSizeOfLeavingEndpoints());
 
-        for (InetAddress host : hosts)
+        for (InetAddressAndPort host : hosts)
         {
-            MessageOut msg = new MessageOut(host, MessagingService.Verb.REPLICATION_FINISHED, null, null, Collections.<String, byte[]>emptyMap());
-            MessagingService.instance().sendRR(msg, FBUtilities.getBroadcastAddress());
+            MessageOut msg = new MessageOut(host, MessagingService.Verb.REPLICATION_FINISHED, null, null, Collections.<Object>emptyList());
+            MessagingService.instance().sendRR(msg, FBUtilities.getBroadcastAddressAndPort());
         }
 
         remover.join();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/SerializationsTest.java b/test/unit/org/apache/cassandra/service/SerializationsTest.java
index d943bb7..c29e7a8 100644
--- a/test/unit/org/apache/cassandra/service/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/service/SerializationsTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.service;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -41,6 +40,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.repair.NodePair;
 import org.apache.cassandra.repair.RepairJobDesc;
@@ -61,6 +61,8 @@ public class SerializationsTest extends AbstractSerializationsTester
     private static Range<Token> FULL_RANGE;
     private static RepairJobDesc DESC;
 
+    private static final int PORT = 7010;
+
     @BeforeClass
     public static void defineSchema() throws Exception
     {
@@ -123,7 +125,7 @@ public class SerializationsTest extends AbstractSerializationsTester
 
         // empty validation
         mt.addMerkleTree((int) Math.pow(2, 15), FULL_RANGE);
-        Validator v0 = new Validator(DESC, FBUtilities.getBroadcastAddress(), -1, PreviewKind.NONE);
+        Validator v0 = new Validator(DESC, FBUtilities.getBroadcastAddressAndPort(), -1, PreviewKind.NONE);
         ValidationComplete c0 = new ValidationComplete(DESC, mt);
 
         // validation with a tree
@@ -131,7 +133,7 @@ public class SerializationsTest extends AbstractSerializationsTester
         mt.addMerkleTree(Integer.MAX_VALUE, FULL_RANGE);
         for (int i = 0; i < 10; i++)
             mt.split(p.getRandomToken());
-        Validator v1 = new Validator(DESC, FBUtilities.getBroadcastAddress(), -1, PreviewKind.NONE);
+        Validator v1 = new Validator(DESC, FBUtilities.getBroadcastAddressAndPort(), -1, PreviewKind.NONE);
         ValidationComplete c1 = new ValidationComplete(DESC, mt);
 
         // validation failed
@@ -180,9 +182,9 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private void testSyncRequestWrite() throws IOException
     {
-        InetAddress local = InetAddress.getByAddress(new byte[]{127, 0, 0, 1});
-        InetAddress src = InetAddress.getByAddress(new byte[]{127, 0, 0, 2});
-        InetAddress dest = InetAddress.getByAddress(new byte[]{127, 0, 0, 3});
+        InetAddressAndPort local = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", PORT);
+        InetAddressAndPort src = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", PORT);
+        InetAddressAndPort dest = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.3", PORT);
 
         SyncRequest message = new SyncRequest(DESC, local, src, dest, Collections.singleton(FULL_RANGE), PreviewKind.NONE);
         testRepairMessageWrite("service.SyncRequest.bin", message);
@@ -194,9 +196,9 @@ public class SerializationsTest extends AbstractSerializationsTester
         if (EXECUTE_WRITES)
             testSyncRequestWrite();
 
-        InetAddress local = InetAddress.getByAddress(new byte[]{127, 0, 0, 1});
-        InetAddress src = InetAddress.getByAddress(new byte[]{127, 0, 0, 2});
-        InetAddress dest = InetAddress.getByAddress(new byte[]{127, 0, 0, 3});
+        InetAddressAndPort local = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", PORT);
+        InetAddressAndPort src = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", PORT);
+        InetAddressAndPort dest = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.3", PORT);
 
         try (DataInputStreamPlus in = getInput("service.SyncRequest.bin"))
         {
@@ -214,8 +216,8 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private void testSyncCompleteWrite() throws IOException
     {
-        InetAddress src = InetAddress.getByAddress(new byte[]{127, 0, 0, 2});
-        InetAddress dest = InetAddress.getByAddress(new byte[]{127, 0, 0, 3});
+        InetAddressAndPort src = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", PORT);
+        InetAddressAndPort dest = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.3", PORT);
         // sync success
         List<SessionSummary> summaries = new ArrayList<>();
         summaries.add(new SessionSummary(src, dest,
@@ -235,8 +237,8 @@ public class SerializationsTest extends AbstractSerializationsTester
         if (EXECUTE_WRITES)
             testSyncCompleteWrite();
 
-        InetAddress src = InetAddress.getByAddress(new byte[]{127, 0, 0, 2});
-        InetAddress dest = InetAddress.getByAddress(new byte[]{127, 0, 0, 3});
+        InetAddressAndPort src = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", PORT);
+        InetAddressAndPort dest = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.3", PORT);
         NodePair nodes = new NodePair(src, dest);
 
         try (DataInputStreamPlus in = getInput("service.SyncComplete.bin"))
@@ -246,6 +248,8 @@ public class SerializationsTest extends AbstractSerializationsTester
             assert message.messageType == RepairMessage.Type.SYNC_COMPLETE;
             assert DESC.equals(message.desc);
 
+            System.out.println(nodes);
+            System.out.println(((SyncComplete) message).nodes);
             assert nodes.equals(((SyncComplete) message).nodes);
             assert ((SyncComplete) message).success;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/service/StorageProxyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageProxyTest.java b/test/unit/org/apache/cassandra/service/StorageProxyTest.java
index bdf45fe..590cfeb 100644
--- a/test/unit/org/apache/cassandra/service/StorageProxyTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageProxyTest.java
@@ -18,7 +18,6 @@
 */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.List;
 
 import org.junit.BeforeClass;
@@ -27,6 +26,7 @@ import org.junit.Test;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 
 import static org.apache.cassandra.Util.rp;
@@ -81,8 +81,8 @@ public class StorageProxyTest
         DatabaseDescriptor.daemonInitialization();
         DatabaseDescriptor.getHintsDirectory().mkdir();
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-        tmd.updateNormalToken(token("1"), InetAddress.getByName("127.0.0.1"));
-        tmd.updateNormalToken(token("6"), InetAddress.getByName("127.0.0.6"));
+        tmd.updateNormalToken(token("1"), InetAddressAndPort.getByName("127.0.0.1"));
+        tmd.updateNormalToken(token("6"), InetAddressAndPort.getByName("127.0.0.6"));
     }
 
     // test getRestrictedRanges for token


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


[04/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
index 522b118..df81a9b 100644
--- a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.locator;
 
 import java.io.IOException;
-import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -32,8 +32,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-
-import com.google.common.net.InetAddresses;
+import java.util.regex.Matcher;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
@@ -61,7 +60,7 @@ public class PropertyFileSnitchTest
     private Path backupFile;
 
     private VersionedValue.VersionedValueFactory valueFactory;
-    private Map<InetAddress, Set<Token>> tokenMap;
+    private Map<InetAddressAndPort, Set<Token>> tokenMap;
 
     @BeforeClass
     public static void setupDD()
@@ -78,17 +77,17 @@ public class PropertyFileSnitchTest
 
         restoreOrigConfigFile();
 
-        InetAddress[] hosts = {
-            InetAddress.getByName("127.0.0.1"), // this exists in the config file
-            InetAddress.getByName("127.0.0.2"), // this exists in the config file
-            InetAddress.getByName("127.0.0.9"), // this does not exist in the config file
+        InetAddressAndPort[] hosts = {
+        InetAddressAndPort.getByName("127.0.0.1"), // this exists in the config file
+        InetAddressAndPort.getByName("127.0.0.2"), // this exists in the config file
+        InetAddressAndPort.getByName("127.0.0.9"), // this does not exist in the config file
         };
 
         IPartitioner partitioner = new RandomPartitioner();
         valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
         tokenMap = new HashMap<>();
 
-        for (InetAddress host : hosts)
+        for (InetAddressAndPort host : hosts)
         {
             Set<Token> tokens = Collections.singleton(partitioner.getRandomToken());
             Gossiper.instance.initializeNodeUnsafe(host, UUID.randomUUID(), 1);
@@ -117,13 +116,21 @@ public class PropertyFileSnitchTest
         for (String line : lines)
         {
             String[] info = line.split("=");
-            if (info.length == 2 && replacements.containsKey(info[0]))
+            if (info.length == 2 && !line.startsWith("#") && !line.startsWith("default="))
             {
-                String replacement = replacements.get(info[0]);
-                if (!replacement.isEmpty()) // empty means remove this line
-                    newLines.add(info[0] + '=' + replacement);
-
-                replaced.add(info[0]);
+                InetAddressAndPort address = InetAddressAndPort.getByName(info[0].replaceAll(Matcher.quoteReplacement("\\:"), ":"));
+                String replacement = replacements.get(address.toString());
+                if (replacement != null)
+                {
+                    if (!replacement.isEmpty()) // empty means remove this line
+                        newLines.add(info[0] + '=' + replacement);
+
+                    replaced.add(address.toString());
+                }
+                else
+                {
+                    newLines.add(line);
+                }
             }
             else
             {
@@ -138,21 +145,26 @@ public class PropertyFileSnitchTest
                 continue;
 
             if (!replacement.getValue().isEmpty()) // empty means remove this line so do nothing here
-                newLines.add(replacement.getKey() + '=' + replacement.getValue());
+            {
+                String escaped = replacement.getKey().replaceAll(Matcher.quoteReplacement(":"), "\\\\:");
+                newLines.add(escaped + '=' + replacement.getValue());
+            }
         }
 
         Files.write(effectiveFile, newLines, StandardCharsets.UTF_8, StandardOpenOption.TRUNCATE_EXISTING);
     }
 
-    private void setNodeShutdown(InetAddress host)
+    private void setNodeShutdown(InetAddressAndPort host)
     {
         StorageService.instance.getTokenMetadata().removeEndpoint(host);
+        Gossiper.instance.injectApplicationState(host, ApplicationState.STATUS_WITH_PORT, valueFactory.shutdown(true));
         Gossiper.instance.injectApplicationState(host, ApplicationState.STATUS, valueFactory.shutdown(true));
         Gossiper.instance.markDead(host, Gossiper.instance.getEndpointStateForEndpoint(host));
     }
 
-    private void setNodeLive(InetAddress host)
+    private void setNodeLive(InetAddressAndPort host)
     {
+        Gossiper.instance.injectApplicationState(host, ApplicationState.STATUS_WITH_PORT, valueFactory.normal(tokenMap.get(host)));
         Gossiper.instance.injectApplicationState(host, ApplicationState.STATUS, valueFactory.normal(tokenMap.get(host)));
         Gossiper.instance.realMarkAlive(host, Gossiper.instance.getEndpointStateForEndpoint(host));
         StorageService.instance.getTokenMetadata().updateNormalTokens(tokenMap.get(host), host);
@@ -160,9 +172,9 @@ public class PropertyFileSnitchTest
 
     private static void checkEndpoint(final AbstractNetworkTopologySnitch snitch,
                                       final String endpointString, final String expectedDatacenter,
-                                      final String expectedRack)
+                                      final String expectedRack) throws UnknownHostException
     {
-        final InetAddress endpoint = InetAddresses.forString(endpointString);
+        final InetAddressAndPort endpoint = InetAddressAndPort.getByName(endpointString);
         assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint));
         assertEquals(expectedRack, snitch.getRack(endpoint));
     }
@@ -174,25 +186,25 @@ public class PropertyFileSnitchTest
     @Test
     public void testChangeHostRack() throws Exception
     {
-        final InetAddress host = InetAddress.getByName("127.0.0.1");
+        final InetAddressAndPort host = InetAddressAndPort.getByName("127.0.0.1");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
+        checkEndpoint(snitch, host.toString(), "DC1", "RAC1");
 
         try
         {
             setNodeLive(host);
 
             Files.copy(effectiveFile, backupFile);
-            replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC1:RAC2"));
+            replaceConfigFile(Collections.singletonMap(host.toString(), "DC1:RAC2"));
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
+            checkEndpoint(snitch, host.toString(), "DC1", "RAC1");
 
             setNodeShutdown(host);
-            replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC1:RAC2"));
+            replaceConfigFile(Collections.singletonMap(host.toString(), "DC1:RAC2"));
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2");
+            checkEndpoint(snitch, host.toString(), "DC1", "RAC2");
         }
         finally
         {
@@ -208,25 +220,25 @@ public class PropertyFileSnitchTest
     @Test
     public void testChangeHostDc() throws Exception
     {
-        final InetAddress host = InetAddress.getByName("127.0.0.1");
+        final InetAddressAndPort host = InetAddressAndPort.getByName("127.0.0.1");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
+        checkEndpoint(snitch, host.toString(), "DC1", "RAC1");
 
         try
         {
             setNodeLive(host);
 
             Files.copy(effectiveFile, backupFile);
-            replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC1"));
+            replaceConfigFile(Collections.singletonMap(host.toString(), "DC2:RAC1"));
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
+            checkEndpoint(snitch, host.toString(), "DC1", "RAC1");
 
             setNodeShutdown(host);
-            replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC1"));
+            replaceConfigFile(Collections.singletonMap(host.toString(), "DC2:RAC1"));
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC1");
+            checkEndpoint(snitch, host.toString(), "DC2", "RAC1");
         }
         finally
         {
@@ -243,25 +255,25 @@ public class PropertyFileSnitchTest
     @Test
     public void testAddHost() throws Exception
     {
-        final InetAddress host = InetAddress.getByName("127.0.0.9");
+        final InetAddressAndPort host = InetAddressAndPort.getByName("127.0.0.9");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
+        checkEndpoint(snitch, host.toString(), "DC1", "r1"); // default
 
         try
         {
             setNodeLive(host);
 
             Files.copy(effectiveFile, backupFile);
-            replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC2")); // add this line if not yet there
+            replaceConfigFile(Collections.singletonMap(host.toString(), "DC2:RAC2")); // add this line if not yet there
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged
+            checkEndpoint(snitch, host.toString(), "DC1", "r1"); // unchanged
 
             setNodeShutdown(host);
-            replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC2")); // add this line if not yet there
+            replaceConfigFile(Collections.singletonMap(host.toString(), "DC2:RAC2")); // add this line if not yet there
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC2"); // changed
+            checkEndpoint(snitch, host.toString(), "DC2", "RAC2"); // changed
         }
         finally
         {
@@ -278,25 +290,25 @@ public class PropertyFileSnitchTest
     @Test
     public void testRemoveHost() throws Exception
     {
-        final InetAddress host = InetAddress.getByName("127.0.0.2");
+        final InetAddressAndPort host = InetAddressAndPort.getByName("127.0.0.2");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2");
+        checkEndpoint(snitch, host.toString(), "DC1", "RAC2");
 
         try
         {
             setNodeLive(host);
 
             Files.copy(effectiveFile, backupFile);
-            replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "")); // removes line if found
+            replaceConfigFile(Collections.singletonMap(host.toString(), "")); // removes line if found
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); // unchanged
+            checkEndpoint(snitch, host.toString(), "DC1", "RAC2"); // unchanged
 
             setNodeShutdown(host);
-            replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "")); // removes line if found
+            replaceConfigFile(Collections.singletonMap(host.toString(), "")); // removes line if found
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
+            checkEndpoint(snitch, host.toString(), "DC1", "r1"); // default
         }
         finally
         {
@@ -313,9 +325,9 @@ public class PropertyFileSnitchTest
     @Test
     public void testChangeDefault() throws Exception
     {
-        final InetAddress host = InetAddress.getByName("127.0.0.9");
+        final InetAddressAndPort host = InetAddressAndPort.getByName("127.0.0.9");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
+        checkEndpoint(snitch, host.toString(), "DC1", "r1"); // default
 
         try
         {
@@ -325,13 +337,13 @@ public class PropertyFileSnitchTest
             replaceConfigFile(Collections.singletonMap("default", "DC2:r2")); // change default
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged
+            checkEndpoint(snitch, host.toString(), "DC1", "r1"); // unchanged
 
             setNodeShutdown(host);
             replaceConfigFile(Collections.singletonMap("default", "DC2:r2")); // change default again (refresh file update)
 
             Thread.sleep(1500);
-            checkEndpoint(snitch, host.getHostAddress(), "DC2", "r2"); // default updated
+            checkEndpoint(snitch, host.toString(), "DC2", "r2"); // default updated
         }
         finally
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java b/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java
index 232865a..b1c3775 100644
--- a/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java
+++ b/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java
@@ -50,7 +50,7 @@ public class ReconnectableSnitchHelperTest
     public void failedAuthentication() throws Exception
     {
         DatabaseDescriptor.setInternodeAuthenticator(MessagingServiceTest.ALLOW_NOTHING_AUTHENTICATOR);
-        InetAddress address = InetAddress.getByName("127.0.0.250");
+        InetAddressAndPort address = InetAddressAndPort.getByName("127.0.0.250");
         //Should tolerate null returns by MS for the connection
         ReconnectableSnitchHelper.reconnect(address, address, null, null);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
index b83194b..a8caa72 100644
--- a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
+++ b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -55,14 +54,14 @@ public class ReplicationStrategyEndpointCacheTest
 
         strategy = getStrategyWithNewTokenMetadata(Keyspace.open(KEYSPACE).getReplicationStrategy(), tmd);
 
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(10)), InetAddress.getByName("127.0.0.1"));
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(20)), InetAddress.getByName("127.0.0.2"));
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(30)), InetAddress.getByName("127.0.0.3"));
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(40)), InetAddress.getByName("127.0.0.4"));
-        //tmd.updateNormalToken(new BigIntegerToken(String.valueOf(50)), InetAddress.getByName("127.0.0.5"));
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(60)), InetAddress.getByName("127.0.0.6"));
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(70)), InetAddress.getByName("127.0.0.7"));
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(80)), InetAddress.getByName("127.0.0.8"));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(10)), InetAddressAndPort.getByName("127.0.0.1"));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(20)), InetAddressAndPort.getByName("127.0.0.2"));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(30)), InetAddressAndPort.getByName("127.0.0.3"));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(40)), InetAddressAndPort.getByName("127.0.0.4"));
+        //tmd.updateNormalToken(new BigIntegerToken(String.valueOf(50)), InetAddressAndPort.getByName("127.0.0.5", null, null));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(60)), InetAddressAndPort.getByName("127.0.0.6"));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(70)), InetAddressAndPort.getByName("127.0.0.7"));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(80)), InetAddressAndPort.getByName("127.0.0.8"));
     }
 
     @Test
@@ -90,31 +89,31 @@ public class ReplicationStrategyEndpointCacheTest
     public void runCacheRespectsTokenChangesTest(Class stratClass, Map<String, String> configOptions) throws Exception
     {
         setup(stratClass, configOptions);
-        ArrayList<InetAddress> initial;
-        ArrayList<InetAddress> endpoints;
+        ArrayList<InetAddressAndPort> initial;
+        ArrayList<InetAddressAndPort> endpoints;
 
         endpoints = strategy.getNaturalEndpoints(searchToken);
         assert endpoints.size() == 5 : StringUtils.join(endpoints, ",");
 
         // test token addition, in DC2 before existing token
         initial = strategy.getNaturalEndpoints(searchToken);
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(35)), InetAddress.getByName("127.0.0.5"));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(35)), InetAddressAndPort.getByName("127.0.0.5"));
         endpoints = strategy.getNaturalEndpoints(searchToken);
         assert endpoints.size() == 5 : StringUtils.join(endpoints, ",");
         assert !endpoints.equals(initial);
 
         // test token removal, newly created token
         initial = strategy.getNaturalEndpoints(searchToken);
-        tmd.removeEndpoint(InetAddress.getByName("127.0.0.5"));
+        tmd.removeEndpoint(InetAddressAndPort.getByName("127.0.0.5"));
         endpoints = strategy.getNaturalEndpoints(searchToken);
         assert endpoints.size() == 5 : StringUtils.join(endpoints, ",");
-        assert !endpoints.contains(InetAddress.getByName("127.0.0.5"));
+        assert !endpoints.contains(InetAddressAndPort.getByName("127.0.0.5"));
         assert !endpoints.equals(initial);
 
         // test token change
         initial = strategy.getNaturalEndpoints(searchToken);
         //move .8 after search token but before other DC3
-        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(25)), InetAddress.getByName("127.0.0.8"));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(25)), InetAddressAndPort.getByName("127.0.0.8"));
         endpoints = strategy.getNaturalEndpoints(searchToken);
         assert endpoints.size() == 5 : StringUtils.join(endpoints, ",");
         assert !endpoints.equals(initial);
@@ -129,7 +128,7 @@ public class ReplicationStrategyEndpointCacheTest
             super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
-        public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+        public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
         {
             assert !called : "calculateNaturalEndpoints was already called, result should have been cached";
             called = true;
@@ -146,7 +145,7 @@ public class ReplicationStrategyEndpointCacheTest
             super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
-        public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+        public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
         {
             assert !called : "calculateNaturalEndpoints was already called, result should have been cached";
             called = true;
@@ -163,7 +162,7 @@ public class ReplicationStrategyEndpointCacheTest
             super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
-        public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+        public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
         {
             assert !called : "calculateNaturalEndpoints was already called, result should have been cached";
             called = true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
index f97a6e5..fe77b0e 100644
--- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -98,22 +97,22 @@ public class SimpleStrategyTest
         {
             tmd = new TokenMetadata();
             strategy = getStrategy(keyspaceName, tmd);
-            List<InetAddress> hosts = new ArrayList<InetAddress>();
+            List<InetAddressAndPort> hosts = new ArrayList<>();
             for (int i = 0; i < endpointTokens.length; i++)
             {
-                InetAddress ep = InetAddress.getByName("127.0.0." + String.valueOf(i + 1));
+                InetAddressAndPort ep = InetAddressAndPort.getByName("127.0.0." + String.valueOf(i + 1));
                 tmd.updateNormalToken(endpointTokens[i], ep);
                 hosts.add(ep);
             }
 
             for (int i = 0; i < keyTokens.length; i++)
             {
-                List<InetAddress> endpoints = strategy.getNaturalEndpoints(keyTokens[i]);
+                List<InetAddressAndPort> endpoints = strategy.getNaturalEndpoints(keyTokens[i]);
                 assertEquals(strategy.getReplicationFactor(), endpoints.size());
-                List<InetAddress> correctEndpoints = new ArrayList<InetAddress>();
+                List<InetAddressAndPort> correctEndpoints = new ArrayList<>();
                 for (int j = 0; j < endpoints.size(); j++)
                     correctEndpoints.add(hosts.get((i + j + 1) % hosts.size()));
-                assertEquals(new HashSet<InetAddress>(correctEndpoints), new HashSet<InetAddress>(endpoints));
+                assertEquals(new HashSet<>(correctEndpoints), new HashSet<>(endpoints));
             }
         }
     }
@@ -135,17 +134,17 @@ public class SimpleStrategyTest
             keyTokens[i] = new BigIntegerToken(String.valueOf(RING_SIZE * 2 * i + RING_SIZE));
         }
 
-        List<InetAddress> hosts = new ArrayList<InetAddress>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         for (int i = 0; i < endpointTokens.length; i++)
         {
-            InetAddress ep = InetAddress.getByName("127.0.0." + String.valueOf(i + 1));
+            InetAddressAndPort ep = InetAddressAndPort.getByName("127.0.0." + String.valueOf(i + 1));
             tmd.updateNormalToken(endpointTokens[i], ep);
             hosts.add(ep);
         }
 
         // bootstrap at the end of the ring
         Token bsToken = new BigIntegerToken(String.valueOf(210));
-        InetAddress bootstrapEndpoint = InetAddress.getByName("127.0.0.11");
+        InetAddressAndPort bootstrapEndpoint = InetAddressAndPort.getByName("127.0.0.11");
         tmd.addBootstrapToken(bsToken, bootstrapEndpoint);
 
         AbstractReplicationStrategy strategy = null;
@@ -159,7 +158,7 @@ public class SimpleStrategyTest
 
             for (int i = 0; i < keyTokens.length; i++)
             {
-                Collection<InetAddress> endpoints = tmd.getWriteEndpoints(keyTokens[i], keyspaceName, strategy.getNaturalEndpoints(keyTokens[i]));
+                Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(keyTokens[i], keyspaceName, strategy.getNaturalEndpoints(keyTokens[i]));
                 assertTrue(endpoints.size() >= replicationFactor);
 
                 for (int j = 0; j < replicationFactor; j++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java b/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
index e5a86fd..b589d2d 100644
--- a/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
+++ b/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Map;
@@ -56,8 +55,8 @@ public class TokenMetadataTest
     {
         DatabaseDescriptor.daemonInitialization();
         tmd = StorageService.instance.getTokenMetadata();
-        tmd.updateNormalToken(token(ONE), InetAddress.getByName("127.0.0.1"));
-        tmd.updateNormalToken(token(SIX), InetAddress.getByName("127.0.0.6"));
+        tmd.updateNormalToken(token(ONE), InetAddressAndPort.getByName("127.0.0.1"));
+        tmd.updateNormalToken(token(SIX), InetAddressAndPort.getByName("127.0.0.6"));
     }
 
     private static void testRingIterator(ArrayList<Token> ring, String start, boolean includeMin, String... expected)
@@ -98,8 +97,8 @@ public class TokenMetadataTest
     @Test
     public void testTopologyUpdate_RackConsolidation() throws UnknownHostException
     {
-        final InetAddress first = InetAddress.getByName("127.0.0.1");
-        final InetAddress second = InetAddress.getByName("127.0.0.6");
+        final InetAddressAndPort first = InetAddressAndPort.getByName("127.0.0.1");
+        final InetAddressAndPort second = InetAddressAndPort.getByName("127.0.0.6");
         final String DATA_CENTER = "datacenter1";
         final String RACK1 = "rack1";
         final String RACK2 = "rack2";
@@ -107,19 +106,19 @@ public class TokenMetadataTest
         DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
         {
             @Override
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 return endpoint.equals(first) ? RACK1 : RACK2;
             }
 
             @Override
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
                 return DATA_CENTER;
             }
 
             @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
             {
                 return 0;
             }
@@ -134,14 +133,14 @@ public class TokenMetadataTest
         TokenMetadata.Topology topology = tokenMetadata.getTopology();
         assertNotNull(topology);
 
-        Multimap<String, InetAddress> allEndpoints = topology.getDatacenterEndpoints();
+        Multimap<String, InetAddressAndPort> allEndpoints = topology.getDatacenterEndpoints();
         assertNotNull(allEndpoints);
         assertTrue(allEndpoints.size() == 2);
         assertTrue(allEndpoints.containsKey(DATA_CENTER));
         assertTrue(allEndpoints.get(DATA_CENTER).contains(first));
         assertTrue(allEndpoints.get(DATA_CENTER).contains(second));
 
-        Map<String, Multimap<String, InetAddress>> racks = topology.getDatacenterRacks();
+        Map<String, Multimap<String, InetAddressAndPort>> racks = topology.getDatacenterRacks();
         assertNotNull(racks);
         assertTrue(racks.size() == 1);
         assertTrue(racks.containsKey(DATA_CENTER));
@@ -154,19 +153,19 @@ public class TokenMetadataTest
         DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
         {
             @Override
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 return RACK1;
             }
 
             @Override
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
                 return DATA_CENTER;
             }
 
             @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
             {
                 return 0;
             }
@@ -196,8 +195,8 @@ public class TokenMetadataTest
     @Test
     public void testTopologyUpdate_RackExpansion() throws UnknownHostException
     {
-        final InetAddress first = InetAddress.getByName("127.0.0.1");
-        final InetAddress second = InetAddress.getByName("127.0.0.6");
+        final InetAddressAndPort first = InetAddressAndPort.getByName("127.0.0.1");
+        final InetAddressAndPort second = InetAddressAndPort.getByName("127.0.0.6");
         final String DATA_CENTER = "datacenter1";
         final String RACK1 = "rack1";
         final String RACK2 = "rack2";
@@ -205,19 +204,19 @@ public class TokenMetadataTest
         DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
         {
             @Override
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 return RACK1;
             }
 
             @Override
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
                 return DATA_CENTER;
             }
 
             @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
             {
                 return 0;
             }
@@ -232,14 +231,14 @@ public class TokenMetadataTest
         TokenMetadata.Topology topology = tokenMetadata.getTopology();
         assertNotNull(topology);
 
-        Multimap<String, InetAddress> allEndpoints = topology.getDatacenterEndpoints();
+        Multimap<String, InetAddressAndPort> allEndpoints = topology.getDatacenterEndpoints();
         assertNotNull(allEndpoints);
         assertTrue(allEndpoints.size() == 2);
         assertTrue(allEndpoints.containsKey(DATA_CENTER));
         assertTrue(allEndpoints.get(DATA_CENTER).contains(first));
         assertTrue(allEndpoints.get(DATA_CENTER).contains(second));
 
-        Map<String, Multimap<String, InetAddress>> racks = topology.getDatacenterRacks();
+        Map<String, Multimap<String, InetAddressAndPort>> racks = topology.getDatacenterRacks();
         assertNotNull(racks);
         assertTrue(racks.size() == 1);
         assertTrue(racks.containsKey(DATA_CENTER));
@@ -252,19 +251,19 @@ public class TokenMetadataTest
         DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
         {
             @Override
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 return endpoint.equals(first) ? RACK1 : RACK2;
             }
 
             @Override
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
                 return DATA_CENTER;
             }
 
             @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
             {
                 return 0;
             }
@@ -293,8 +292,8 @@ public class TokenMetadataTest
     @Test
     public void testEndpointSizes() throws UnknownHostException
     {
-        final InetAddress first = InetAddress.getByName("127.0.0.1");
-        final InetAddress second = InetAddress.getByName("127.0.0.6");
+        final InetAddressAndPort first = InetAddressAndPort.getByName("127.0.0.1");
+        final InetAddressAndPort second = InetAddressAndPort.getByName("127.0.0.6");
 
         tmd.updateNormalToken(token(ONE), first);
         tmd.updateNormalToken(token(SIX), second);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java b/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
index 2394e0c..2b13715 100644
--- a/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
@@ -20,7 +20,6 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.net.InetAddress;
 import java.util.Map;
 import java.util.UUID;
 
@@ -35,6 +34,7 @@ import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.hints.HintsService;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static org.junit.Assert.assertEquals;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
@@ -53,11 +53,15 @@ public class HintedHandOffMetricsTest
         DatabaseDescriptor.getHintsDirectory().mkdirs();
 
         for (int i = 0; i < 99; i++)
-            HintsService.instance.metrics.incrPastWindow(InetAddress.getByName("127.0.0.1"));
+            HintsService.instance.metrics.incrPastWindow(InetAddressAndPort.getLocalHost());
         HintsService.instance.metrics.log();
 
-        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS);
+        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS_V2);
         Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
         assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
+
+        rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.LEGACY_PEER_EVENTS);
+        returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
+        assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/net/CompactEndpointSerializationHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/CompactEndpointSerializationHelperTest.java b/test/unit/org/apache/cassandra/net/CompactEndpointSerializationHelperTest.java
new file mode 100644
index 0000000..3232455
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/CompactEndpointSerializationHelperTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+import static org.junit.Assert.assertEquals;
+
+public class CompactEndpointSerializationHelperTest
+{
+
+    @Test
+    public void testRoundtrip() throws Exception
+    {
+        InetAddressAndPort ipv4 = InetAddressAndPort.getByName("127.0.0.1:42");
+        InetAddressAndPort ipv6 = InetAddressAndPort.getByName("[2001:db8:0:0:0:ff00:42:8329]:42");
+
+        testAddress(ipv4, MessagingService.VERSION_30);
+        testAddress(ipv6, MessagingService.VERSION_30);
+        testAddress(ipv4, MessagingService.current_version);
+        testAddress(ipv6, MessagingService.current_version);
+    }
+
+    private void testAddress(InetAddressAndPort address, int version) throws Exception
+    {
+        ByteBuffer out;
+        try (DataOutputBuffer dob = new DataOutputBuffer())
+        {
+            CompactEndpointSerializationHelper.instance.serialize(address, dob, version);
+            out = dob.buffer();
+        }
+        assertEquals(out.remaining(), CompactEndpointSerializationHelper.instance.serializedSize(address, version));
+
+        InetAddressAndPort roundtripped;
+        try (DataInputBuffer dib = new DataInputBuffer(out, false))
+        {
+            roundtripped = CompactEndpointSerializationHelper.instance.deserialize(dib, version);
+        }
+
+        if (version >= MessagingService.VERSION_40)
+        {
+            assertEquals(address, roundtripped);
+        }
+        else
+        {
+            assertEquals(roundtripped.address, address.address);
+            assertEquals(7000, roundtripped.port);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/net/ForwardToContainerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/ForwardToContainerTest.java b/test/unit/org/apache/cassandra/net/ForwardToContainerTest.java
new file mode 100644
index 0000000..195d734
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/ForwardToContainerTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class ForwardToContainerTest
+{
+    @Test
+    public void testCurrent() throws Exception
+    {
+        testVersion(MessagingService.current_version);
+    }
+
+    @Test
+    public void test30() throws Exception
+    {
+        testVersion(MessagingService.VERSION_30);
+    }
+
+    private void testVersion(int version) throws Exception
+    {
+        InetAddressAndPort.initializeDefaultPort(65532);
+        List<InetAddressAndPort> addresses = ImmutableList.of(InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", 42),
+                                                              InetAddressAndPort.getByName("127.0.0.1"),
+                                                              InetAddressAndPort.getByName("127.0.0.1:7000"),
+                                                              InetAddressAndPort.getByNameOverrideDefaults("2001:0db8:0000:0000:0000:ff00:0042:8329", 42),
+                                                              InetAddressAndPort.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329"),
+                                                              InetAddressAndPort.getByName("[2001:0db8:0000:0000:0000:ff00:0042:8329]:7000"));
+
+        ForwardToContainer ftc = new ForwardToContainer(addresses, new int[] { 44, 45, 46, 47, 48, 49 });
+        ByteBuffer buffer;
+        try (DataOutputBuffer dob = new DataOutputBuffer())
+        {
+            ForwardToSerializer.instance.serialize(ftc, dob, version);
+            buffer = dob.buffer();
+        }
+
+        assertEquals(buffer.remaining(), ForwardToSerializer.instance.serializedSize(ftc, version));
+
+        ForwardToContainer deserialized;
+        try (DataInputBuffer dib = new DataInputBuffer(buffer, false))
+        {
+            deserialized = ForwardToSerializer.instance.deserialize(dib, version);
+        }
+
+        assertTrue(Arrays.equals(ftc.messageIds, deserialized.messageIds));
+
+        Iterator<InetAddressAndPort> iterator = deserialized.targets.iterator();
+        if (version >= MessagingService.VERSION_40)
+        {
+            for (int ii = 0; ii < addresses.size(); ii++)
+            {
+                InetAddressAndPort original = addresses.get(ii);
+                InetAddressAndPort roundtripped = iterator.next();
+                assertEquals(original, roundtripped);
+            }
+        }
+        else
+        {
+            for (int ii = 0; ii < addresses.size(); ii++)
+            {
+                InetAddressAndPort original = addresses.get(ii);
+                InetAddressAndPort roundtripped = iterator.next();
+                assertEquals(original.address, roundtripped.address);
+                //3.0 can't send port numbers so you get the defaults
+                assertEquals(65532, roundtripped.port);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/net/Matcher.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/Matcher.java b/test/unit/org/apache/cassandra/net/Matcher.java
index cd1b667..27b685f 100644
--- a/test/unit/org/apache/cassandra/net/Matcher.java
+++ b/test/unit/org/apache/cassandra/net/Matcher.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Predicate based on intercepted, outgoing messange and the message's destination address.
@@ -28,5 +28,5 @@ public interface Matcher<T>
      * @param obj intercepted outgoing message
      * @param to  destination address
      */
-    public boolean matches(MessageOut<T> obj, InetAddress to);
+    public boolean matches(MessageOut<T> obj, InetAddressAndPort to);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/net/MatcherResponse.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/MatcherResponse.java b/test/unit/org/apache/cassandra/net/MatcherResponse.java
index 6cd8085..7a1772a 100644
--- a/test/unit/org/apache/cassandra/net/MatcherResponse.java
+++ b/test/unit/org/apache/cassandra/net/MatcherResponse.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Queue;
@@ -27,6 +26,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 /**
  * Sends a response for an incoming message with a matching {@link Matcher}.
  * The actual behavior by any instance of this class can be inspected by
@@ -76,7 +77,7 @@ public class MatcherResponse
      * Respond with the message created by the provided function that will be called with each intercepted outbound message.
      * @param fnResponse    function to call for creating reply based on intercepted message and target address
      */
-    public <T, S> MockMessagingSpy respond(BiFunction<MessageOut<T>, InetAddress, MessageIn<S>> fnResponse)
+    public <T, S> MockMessagingSpy respond(BiFunction<MessageOut<T>, InetAddressAndPort, MessageIn<S>> fnResponse)
     {
         return respondN(fnResponse, Integer.MAX_VALUE);
     }
@@ -101,7 +102,7 @@ public class MatcherResponse
      */
     public <T, S> MockMessagingSpy respondNWithPayloadForEachReceiver(Function<MessageOut<T>, S> fnResponse, MessagingService.Verb verb, int limit)
     {
-        return respondN((MessageOut<T> msg, InetAddress to) -> {
+        return respondN((MessageOut<T> msg, InetAddressAndPort to) -> {
                     S payload = fnResponse.apply(msg);
                     if (payload == null)
                         return null;
@@ -147,7 +148,7 @@ public class MatcherResponse
      * each intercepted outbound message.
      * @param fnResponse    function to call for creating reply based on intercepted message and target address
      */
-    public <T, S> MockMessagingSpy respondN(BiFunction<MessageOut<T>, InetAddress, MessageIn<S>> fnResponse, int limit)
+    public <T, S> MockMessagingSpy respondN(BiFunction<MessageOut<T>, InetAddressAndPort, MessageIn<S>> fnResponse, int limit)
     {
         limitCounter.set(limit);
 
@@ -155,7 +156,7 @@ public class MatcherResponse
 
         sink = new IMessageSink()
         {
-            public boolean allowOutgoingMessage(MessageOut message, int id, InetAddress to)
+            public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to)
             {
                 // prevent outgoing message from being send in case matcher indicates a match
                 // and instead send the mocked response

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 f0a959e..4ce3422 100644
--- a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
+++ b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
@@ -48,6 +48,10 @@ import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService.ServerChannel;
 import org.apache.cassandra.net.async.NettyFactory;
 import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
@@ -81,7 +85,7 @@ public class MessagingServiceTest
     };
     private static IInternodeAuthenticator originalAuthenticator;
     private static ServerEncryptionOptions originalServerEncryptionOptions;
-    private static InetAddress originalListenAddress;
+    private static InetAddressAndPort originalListenAddress;
 
     private final MessagingService messagingService = MessagingService.test();
 
@@ -93,7 +97,7 @@ public class MessagingServiceTest
         DatabaseDescriptor.setBroadcastAddress(InetAddress.getByName("127.0.0.1"));
         originalAuthenticator = DatabaseDescriptor.getInternodeAuthenticator();
         originalServerEncryptionOptions = DatabaseDescriptor.getServerEncryptionOptions();
-        originalListenAddress = DatabaseDescriptor.getListenAddress();
+        originalListenAddress = InetAddressAndPort.getByAddressOverrideDefaults(DatabaseDescriptor.getListenAddress(), DatabaseDescriptor.getStoragePort());
     }
 
     private static int metricScopeId = 0;
@@ -103,8 +107,8 @@ public class MessagingServiceTest
     {
         messagingService.resetDroppedMessagesMap(Integer.toString(metricScopeId++));
         MockBackPressureStrategy.applied = false;
-        messagingService.destroyConnectionPool(InetAddress.getByName("127.0.0.2"));
-        messagingService.destroyConnectionPool(InetAddress.getByName("127.0.0.3"));
+        messagingService.destroyConnectionPool(InetAddressAndPort.getByName("127.0.0.2"));
+        messagingService.destroyConnectionPool(InetAddressAndPort.getByName("127.0.0.3"));
     }
 
     @After
@@ -113,7 +117,7 @@ public class MessagingServiceTest
         DatabaseDescriptor.setInternodeAuthenticator(originalAuthenticator);
         DatabaseDescriptor.setServerEncryptionOptions(originalServerEncryptionOptions);
         DatabaseDescriptor.setShouldListenOnBroadcastAddress(false);
-        DatabaseDescriptor.setListenAddress(originalListenAddress);
+        DatabaseDescriptor.setListenAddress(originalListenAddress.address);
         FBUtilities.reset();
     }
 
@@ -221,44 +225,44 @@ public class MessagingServiceTest
     @Test
     public void testUpdatesBackPressureOnSendWhenEnabledAndWithSupportedCallback() throws UnknownHostException
     {
-        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddress.getByName("127.0.0.2"));
+        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddressAndPort.getByName("127.0.0.2"));
         IAsyncCallback bpCallback = new BackPressureCallback();
         IAsyncCallback noCallback = new NoBackPressureCallback();
         MessageOut<?> ignored = null;
 
         DatabaseDescriptor.setBackPressureEnabled(true);
-        messagingService.updateBackPressureOnSend(InetAddress.getByName("127.0.0.2"), noCallback, ignored);
+        messagingService.updateBackPressureOnSend(InetAddressAndPort.getByName("127.0.0.2"), noCallback, ignored);
         assertFalse(backPressureState.onSend);
 
         DatabaseDescriptor.setBackPressureEnabled(false);
-        messagingService.updateBackPressureOnSend(InetAddress.getByName("127.0.0.2"), bpCallback, ignored);
+        messagingService.updateBackPressureOnSend(InetAddressAndPort.getByName("127.0.0.2"), bpCallback, ignored);
         assertFalse(backPressureState.onSend);
 
         DatabaseDescriptor.setBackPressureEnabled(true);
-        messagingService.updateBackPressureOnSend(InetAddress.getByName("127.0.0.2"), bpCallback, ignored);
+        messagingService.updateBackPressureOnSend(InetAddressAndPort.getByName("127.0.0.2"), bpCallback, ignored);
         assertTrue(backPressureState.onSend);
     }
 
     @Test
     public void testUpdatesBackPressureOnReceiveWhenEnabledAndWithSupportedCallback() throws UnknownHostException
     {
-        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddress.getByName("127.0.0.2"));
+        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddressAndPort.getByName("127.0.0.2"));
         IAsyncCallback bpCallback = new BackPressureCallback();
         IAsyncCallback noCallback = new NoBackPressureCallback();
         boolean timeout = false;
 
         DatabaseDescriptor.setBackPressureEnabled(true);
-        messagingService.updateBackPressureOnReceive(InetAddress.getByName("127.0.0.2"), noCallback, timeout);
+        messagingService.updateBackPressureOnReceive(InetAddressAndPort.getByName("127.0.0.2"), noCallback, timeout);
         assertFalse(backPressureState.onReceive);
         assertFalse(backPressureState.onTimeout);
 
         DatabaseDescriptor.setBackPressureEnabled(false);
-        messagingService.updateBackPressureOnReceive(InetAddress.getByName("127.0.0.2"), bpCallback, timeout);
+        messagingService.updateBackPressureOnReceive(InetAddressAndPort.getByName("127.0.0.2"), bpCallback, timeout);
         assertFalse(backPressureState.onReceive);
         assertFalse(backPressureState.onTimeout);
 
         DatabaseDescriptor.setBackPressureEnabled(true);
-        messagingService.updateBackPressureOnReceive(InetAddress.getByName("127.0.0.2"), bpCallback, timeout);
+        messagingService.updateBackPressureOnReceive(InetAddressAndPort.getByName("127.0.0.2"), bpCallback, timeout);
         assertTrue(backPressureState.onReceive);
         assertFalse(backPressureState.onTimeout);
     }
@@ -266,23 +270,23 @@ public class MessagingServiceTest
     @Test
     public void testUpdatesBackPressureOnTimeoutWhenEnabledAndWithSupportedCallback() throws UnknownHostException
     {
-        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddress.getByName("127.0.0.2"));
+        MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddressAndPort.getByName("127.0.0.2"));
         IAsyncCallback bpCallback = new BackPressureCallback();
         IAsyncCallback noCallback = new NoBackPressureCallback();
         boolean timeout = true;
 
         DatabaseDescriptor.setBackPressureEnabled(true);
-        messagingService.updateBackPressureOnReceive(InetAddress.getByName("127.0.0.2"), noCallback, timeout);
+        messagingService.updateBackPressureOnReceive(InetAddressAndPort.getByName("127.0.0.2"), noCallback, timeout);
         assertFalse(backPressureState.onReceive);
         assertFalse(backPressureState.onTimeout);
 
         DatabaseDescriptor.setBackPressureEnabled(false);
-        messagingService.updateBackPressureOnReceive(InetAddress.getByName("127.0.0.2"), bpCallback, timeout);
+        messagingService.updateBackPressureOnReceive(InetAddressAndPort.getByName("127.0.0.2"), bpCallback, timeout);
         assertFalse(backPressureState.onReceive);
         assertFalse(backPressureState.onTimeout);
 
         DatabaseDescriptor.setBackPressureEnabled(true);
-        messagingService.updateBackPressureOnReceive(InetAddress.getByName("127.0.0.2"), bpCallback, timeout);
+        messagingService.updateBackPressureOnReceive(InetAddressAndPort.getByName("127.0.0.2"), bpCallback, timeout);
         assertFalse(backPressureState.onReceive);
         assertTrue(backPressureState.onTimeout);
     }
@@ -291,11 +295,11 @@ public class MessagingServiceTest
     public void testAppliesBackPressureWhenEnabled() throws UnknownHostException
     {
         DatabaseDescriptor.setBackPressureEnabled(false);
-        messagingService.applyBackPressure(Arrays.asList(InetAddress.getByName("127.0.0.2")), ONE_SECOND);
+        messagingService.applyBackPressure(Arrays.asList(InetAddressAndPort.getByName("127.0.0.2")), ONE_SECOND);
         assertFalse(MockBackPressureStrategy.applied);
 
         DatabaseDescriptor.setBackPressureEnabled(true);
-        messagingService.applyBackPressure(Arrays.asList(InetAddress.getByName("127.0.0.2")), ONE_SECOND);
+        messagingService.applyBackPressure(Arrays.asList(InetAddressAndPort.getByName("127.0.0.2")), ONE_SECOND);
         assertTrue(MockBackPressureStrategy.applied);
     }
 
@@ -303,13 +307,13 @@ public class MessagingServiceTest
     public void testDoesntApplyBackPressureToBroadcastAddress() throws UnknownHostException
     {
         DatabaseDescriptor.setBackPressureEnabled(true);
-        messagingService.applyBackPressure(Arrays.asList(InetAddress.getByName("127.0.0.1")), ONE_SECOND);
+        messagingService.applyBackPressure(Arrays.asList(InetAddressAndPort.getByName("127.0.0.1")), ONE_SECOND);
         assertFalse(MockBackPressureStrategy.applied);
     }
 
     private static void addDCLatency(long sentAt, long nowTime) throws IOException
     {
-        MessageIn.deriveConstructionTime(InetAddress.getLocalHost(), (int)sentAt, nowTime);
+        MessageIn.deriveConstructionTime(InetAddressAndPort.getLocalHost(), (int)sentAt, nowTime);
     }
 
     public static class MockBackPressureStrategy implements BackPressureStrategy<MockBackPressureStrategy.MockBackPressureState>
@@ -328,19 +332,19 @@ public class MessagingServiceTest
         }
 
         @Override
-        public MockBackPressureState newState(InetAddress host)
+        public MockBackPressureState newState(InetAddressAndPort host)
         {
             return new MockBackPressureState(host);
         }
 
         public static class MockBackPressureState implements BackPressureState
         {
-            private final InetAddress host;
+            private final InetAddressAndPort host;
             public volatile boolean onSend = false;
             public volatile boolean onReceive = false;
             public volatile boolean onTimeout = false;
 
-            private MockBackPressureState(InetAddress host)
+            private MockBackPressureState(InetAddressAndPort host)
             {
                 this.host = host;
             }
@@ -370,7 +374,7 @@ public class MessagingServiceTest
             }
 
             @Override
-            public InetAddress getHost()
+            public InetAddressAndPort getHost()
             {
                 return host;
             }
@@ -429,7 +433,7 @@ public class MessagingServiceTest
     {
         MessagingService ms = MessagingService.instance();
         DatabaseDescriptor.setInternodeAuthenticator(ALLOW_NOTHING_AUTHENTICATOR);
-        InetAddress address = InetAddress.getByName("127.0.0.250");
+        InetAddressAndPort address = InetAddressAndPort.getByName("127.0.0.250");
 
         //Should return null
         MessageOut messageOut = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK);
@@ -444,20 +448,20 @@ public class MessagingServiceTest
     public void testOutboundMessagingConnectionCleansUp() throws Exception
     {
         MessagingService ms = MessagingService.instance();
-        InetSocketAddress local = new InetSocketAddress("127.0.0.1", 9876);
-        InetSocketAddress remote = new InetSocketAddress("127.0.0.2", 9876);
+        InetAddressAndPort local = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", 9876);
+        InetAddressAndPort remote = InetAddressAndPort.getByNameOverrideDefaults("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);
+        OutboundMessagingPool pool = new OutboundMessagingPool(remote, local, null, new MockBackPressureStrategy(null).newState(remote), ALLOW_NOTHING_AUTHENTICATOR);
+        ms.channelManagers.put(remote, pool);
         pool.sendMessage(new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK), 0);
-        assertFalse(ms.channelManagers.containsKey(remote.getAddress()));
+        assertFalse(ms.channelManagers.containsKey(remote));
     }
 
     @Test
-    public void reconnectWithNewIp() throws UnknownHostException
+    public void reconnectWithNewIp() throws Exception
     {
-        InetAddress publicIp = InetAddress.getByName("127.0.0.2");
-        InetAddress privateIp = InetAddress.getByName("127.0.0.3");
+        InetAddressAndPort publicIp = InetAddressAndPort.getByName("127.0.0.2");
+        InetAddressAndPort privateIp = InetAddressAndPort.getByName("127.0.0.3");
 
         // reset the preferred IP value, for good test hygene
         SystemKeyspace.updatePreferredIP(publicIp, publicIp);
@@ -485,8 +489,8 @@ public class MessagingServiceTest
                 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);
+            InetAddressAndPort server = FBUtilities.getBroadcastAddressAndPort();
+            OutboundConnectionIdentifier id = OutboundConnectionIdentifier.small(InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", 0), server);
 
             CountDownLatch latch = new CountDownLatch(1);
             OutboundConnectionParams params = OutboundConnectionParams.builder()
@@ -596,7 +600,7 @@ public class MessagingServiceTest
         if (listenOnBroadcastAddr)
         {
             DatabaseDescriptor.setShouldListenOnBroadcastAddress(true);
-            listenAddress = InetAddresses.increment(FBUtilities.getBroadcastAddress());
+            listenAddress = InetAddresses.increment(FBUtilities.getBroadcastAddressAndPort().address);
             DatabaseDescriptor.setListenAddress(listenAddress);
             FBUtilities.reset();
         }
@@ -627,7 +631,7 @@ public class MessagingServiceTest
 
                     if (serverEncryptionOptions.enable_legacy_ssl_storage_port)
                     {
-                        if (legacySslPort == serverChannel.getAddress().getPort())
+                        if (legacySslPort == serverChannel.getAddress().port)
                         {
                             foundLegacyListenSslAddress = true;
                             Assert.assertEquals(ServerChannel.SecurityLevel.REQUIRED, serverChannel.getSecurityLevel());
@@ -646,7 +650,7 @@ public class MessagingServiceTest
                 int found = 0;
                 for (ServerChannel serverChannel : messagingService.serverChannels)
                 {
-                    if (serverChannel.getAddress().getAddress().equals(listenAddress))
+                    if (serverChannel.getAddress().address.equals(listenAddress))
                         found++;
                 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/net/MockMessagingService.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/MockMessagingService.java b/test/unit/org/apache/cassandra/net/MockMessagingService.java
index 0412759..4ea1bf5 100644
--- a/test/unit/org/apache/cassandra/net/MockMessagingService.java
+++ b/test/unit/org/apache/cassandra/net/MockMessagingService.java
@@ -17,14 +17,15 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.function.Predicate;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 /**
  * Starting point for mocking {@link MessagingService} interactions. Outgoing messages can be
  * intercepted by first creating a {@link MatcherResponse} by calling {@link MockMessagingService#when(Matcher)}.
- * Alternatively {@link Matcher}s can be created by using helper methods such as {@link #to(InetAddress)},
+ * Alternatively {@link Matcher}s can be created by using helper methods such as {@link #to(InetAddressAndPort)},
  * {@link #verb(MessagingService.Verb)} or {@link #payload(Predicate)} and may also be
  * nested using {@link MockMessagingService#all(Matcher[])} or {@link MockMessagingService#any(Matcher[])}.
  * After each test, {@link MockMessagingService#cleanup()} must be called for free listeners registered
@@ -58,11 +59,11 @@ public class MockMessagingService
      * Creates a matcher that will indicate if the target address of the outgoing message equals the
      * provided address.
      */
-    public static Matcher<InetAddress> to(String address)
+    public static Matcher<InetAddressAndPort> to(String address)
     {
         try
         {
-            return to(InetAddress.getByName(address));
+            return to(InetAddressAndPort.getByName(address));
         }
         catch (UnknownHostException e)
         {
@@ -74,7 +75,7 @@ public class MockMessagingService
      * Creates a matcher that will indicate if the target address of the outgoing message equals the
      * provided address.
      */
-    public static Matcher<InetAddress> to(InetAddress address)
+    public static Matcher<InetAddressAndPort> to(InetAddressAndPort address)
     {
         return (in, to) -> to == address || to.equals(address);
     }
@@ -117,7 +118,7 @@ public class MockMessagingService
      */
     public static <T> Matcher<?> all(Matcher<?>... matchers)
     {
-        return (MessageOut<T> out, InetAddress to) -> {
+        return (MessageOut<T> out, InetAddressAndPort to) -> {
             for (Matcher matcher : matchers)
             {
                 if (!matcher.matches(out, to))
@@ -132,7 +133,7 @@ public class MockMessagingService
      */
     public static <T> Matcher<?> any(Matcher<?>... matchers)
     {
-        return (MessageOut<T> out, InetAddress to) -> {
+        return (MessageOut<T> out, InetAddressAndPort to) -> {
             for (Matcher matcher : matchers)
             {
                 if (matcher.matches(out, to))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java b/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java
index 3f6564e..8d0f91b 100644
--- a/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java
+++ b/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java
@@ -55,7 +55,7 @@ public class MockMessagingServiceTest
     public void testRequestResponse() throws InterruptedException, ExecutionException
     {
         // echo message that we like to mock as incoming reply for outgoing echo message
-        MessageIn<EchoMessage> echoMessageIn = MessageIn.create(FBUtilities.getBroadcastAddress(),
+        MessageIn<EchoMessage> echoMessageIn = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(),
                 EchoMessage.instance,
                 Collections.emptyMap(),
                 MessagingService.Verb.ECHO,
@@ -63,14 +63,14 @@ public class MockMessagingServiceTest
         MockMessagingSpy spy = MockMessagingService
                 .when(
                         all(
-                                to(FBUtilities.getBroadcastAddress()),
+                                to(FBUtilities.getBroadcastAddressAndPort()),
                                 verb(MessagingService.Verb.ECHO)
                         )
                 )
                 .respond(echoMessageIn);
 
         MessageOut<EchoMessage> echoMessageOut = new MessageOut<>(MessagingService.Verb.ECHO, EchoMessage.instance, EchoMessage.serializer);
-        MessagingService.instance().sendRR(echoMessageOut, FBUtilities.getBroadcastAddress(), new IAsyncCallback()
+        MessagingService.instance().sendRR(echoMessageOut, FBUtilities.getBroadcastAddressAndPort(), new IAsyncCallback()
         {
             public void response(MessageIn msg)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/net/RateBasedBackPressureTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/RateBasedBackPressureTest.java b/test/unit/org/apache/cassandra/net/RateBasedBackPressureTest.java
index 93fe32e..4d1ae01 100644
--- a/test/unit/org/apache/cassandra/net/RateBasedBackPressureTest.java
+++ b/test/unit/org/apache/cassandra/net/RateBasedBackPressureTest.java
@@ -18,7 +18,6 @@
 */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -29,6 +28,7 @@ import com.google.common.util.concurrent.RateLimiter;
 
 import org.junit.Test;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.TestTimeSource;
 import org.apache.cassandra.utils.TimeSource;
 
@@ -94,17 +94,17 @@ public class RateBasedBackPressureTest
         TestTimeSource timeSource = new TestTimeSource();
         RateBasedBackPressure strategy = new RateBasedBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "FAST"), timeSource, windowSize);
 
-        RateBasedBackPressureState state = strategy.newState(InetAddress.getLoopbackAddress());
+        RateBasedBackPressureState state = strategy.newState(InetAddressAndPort.getLoopbackAddress());
         state.onMessageSent(null);
         assertEquals(0, state.incomingRate.size());
         assertEquals(0, state.outgoingRate.size());
 
-        state = strategy.newState(InetAddress.getLoopbackAddress());
+        state = strategy.newState(InetAddressAndPort.getLoopbackAddress());
         state.onResponseReceived();
         assertEquals(1, state.incomingRate.size());
         assertEquals(1, state.outgoingRate.size());
 
-        state = strategy.newState(InetAddress.getLoopbackAddress());
+        state = strategy.newState(InetAddressAndPort.getLoopbackAddress());
         state.onResponseTimeout();
         assertEquals(0, state.incomingRate.size());
         assertEquals(1, state.outgoingRate.size());
@@ -116,7 +116,7 @@ public class RateBasedBackPressureTest
         long windowSize = 6000;
         TestTimeSource timeSource = new TestTimeSource();
         RateBasedBackPressure strategy = new RateBasedBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "FAST"), timeSource, windowSize);
-        RateBasedBackPressureState state = strategy.newState(InetAddress.getLoopbackAddress());
+        RateBasedBackPressureState state = strategy.newState(InetAddressAndPort.getLoopbackAddress());
 
         // Get initial rate:
         double initialRate = state.rateLimiter.getRate();
@@ -140,7 +140,7 @@ public class RateBasedBackPressureTest
         long windowSize = 6000;
         TestTimeSource timeSource = new TestTimeSource();
         RateBasedBackPressure strategy = new RateBasedBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "FAST"), timeSource, windowSize);
-        RateBasedBackPressureState state = strategy.newState(InetAddress.getLoopbackAddress());
+        RateBasedBackPressureState state = strategy.newState(InetAddressAndPort.getLoopbackAddress());
 
         // Get initial time:
         long current = state.getLastIntervalAcquire();
@@ -174,7 +174,7 @@ public class RateBasedBackPressureTest
         long windowSize = 6000;
         TestTimeSource timeSource = new TestTimeSource();
         RateBasedBackPressure strategy = new RateBasedBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "FAST"), timeSource, windowSize);
-        RateBasedBackPressureState state = strategy.newState(InetAddress.getLoopbackAddress());
+        RateBasedBackPressureState state = strategy.newState(InetAddressAndPort.getLoopbackAddress());
 
         // Update incoming and outgoing rate so that the ratio is 0.5:
         state.incomingRate.update(50);
@@ -194,7 +194,7 @@ public class RateBasedBackPressureTest
         long windowSize = 6000;
         TestTimeSource timeSource = new TestTimeSource();
         RateBasedBackPressure strategy = new RateBasedBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "FAST"), timeSource, windowSize);
-        RateBasedBackPressureState state = strategy.newState(InetAddress.getLoopbackAddress());
+        RateBasedBackPressureState state = strategy.newState(InetAddressAndPort.getLoopbackAddress());
 
         // Update incoming and outgoing rate so that the ratio is 0.5:
         state.incomingRate.update(50);
@@ -236,9 +236,9 @@ public class RateBasedBackPressureTest
         long windowSize = 6000;
         TestTimeSource timeSource = new TestTimeSource();
         TestableBackPressure strategy = new TestableBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "FAST"), timeSource, windowSize);
-        RateBasedBackPressureState state1 = strategy.newState(InetAddress.getByName("127.0.0.1"));
-        RateBasedBackPressureState state2 = strategy.newState(InetAddress.getByName("127.0.0.2"));
-        RateBasedBackPressureState state3 = strategy.newState(InetAddress.getByName("127.0.0.3"));
+        RateBasedBackPressureState state1 = strategy.newState(InetAddressAndPort.getByName("127.0.0.1"));
+        RateBasedBackPressureState state2 = strategy.newState(InetAddressAndPort.getByName("127.0.0.2"));
+        RateBasedBackPressureState state3 = strategy.newState(InetAddressAndPort.getByName("127.0.0.3"));
 
         // Update incoming and outgoing rates:
         state1.incomingRate.update(50);
@@ -265,9 +265,9 @@ public class RateBasedBackPressureTest
         long windowSize = 6000;
         TestTimeSource timeSource = new TestTimeSource();
         TestableBackPressure strategy = new TestableBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "SLOW"), timeSource, windowSize);
-        RateBasedBackPressureState state1 = strategy.newState(InetAddress.getByName("127.0.0.1"));
-        RateBasedBackPressureState state2 = strategy.newState(InetAddress.getByName("127.0.0.2"));
-        RateBasedBackPressureState state3 = strategy.newState(InetAddress.getByName("127.0.0.3"));
+        RateBasedBackPressureState state1 = strategy.newState(InetAddressAndPort.getByName("127.0.0.1"));
+        RateBasedBackPressureState state2 = strategy.newState(InetAddressAndPort.getByName("127.0.0.2"));
+        RateBasedBackPressureState state3 = strategy.newState(InetAddressAndPort.getByName("127.0.0.3"));
 
         // Update incoming and outgoing rates:
         state1.incomingRate.update(50);
@@ -294,10 +294,10 @@ public class RateBasedBackPressureTest
         long windowSize = 6000;
         TestTimeSource timeSource = new TestTimeSource();
         TestableBackPressure strategy = new TestableBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "SLOW"), timeSource, windowSize);
-        RateBasedBackPressureState state1 = strategy.newState(InetAddress.getByName("127.0.0.1"));
-        RateBasedBackPressureState state2 = strategy.newState(InetAddress.getByName("127.0.0.2"));
-        RateBasedBackPressureState state3 = strategy.newState(InetAddress.getByName("127.0.0.3"));
-        RateBasedBackPressureState state4 = strategy.newState(InetAddress.getByName("127.0.0.4"));
+        RateBasedBackPressureState state1 = strategy.newState(InetAddressAndPort.getByName("127.0.0.1"));
+        RateBasedBackPressureState state2 = strategy.newState(InetAddressAndPort.getByName("127.0.0.2"));
+        RateBasedBackPressureState state3 = strategy.newState(InetAddressAndPort.getByName("127.0.0.3"));
+        RateBasedBackPressureState state4 = strategy.newState(InetAddressAndPort.getByName("127.0.0.4"));
 
         // Update incoming and outgoing rates:
         state1.incomingRate.update(50); // this
@@ -333,9 +333,9 @@ public class RateBasedBackPressureTest
         long windowSize = 10000;
         TestTimeSource timeSource = new TestTimeSource();
         TestableBackPressure strategy = new TestableBackPressure(ImmutableMap.of(HIGH_RATIO, "0.9", FACTOR, "10", FLOW, "SLOW"), timeSource, windowSize);
-        RateBasedBackPressureState state1 = strategy.newState(InetAddress.getByName("127.0.0.1"));
-        RateBasedBackPressureState state2 = strategy.newState(InetAddress.getByName("127.0.0.2"));
-        RateBasedBackPressureState state3 = strategy.newState(InetAddress.getByName("127.0.0.3"));
+        RateBasedBackPressureState state1 = strategy.newState(InetAddressAndPort.getByName("127.0.0.1"));
+        RateBasedBackPressureState state2 = strategy.newState(InetAddressAndPort.getByName("127.0.0.2"));
+        RateBasedBackPressureState state3 = strategy.newState(InetAddressAndPort.getByName("127.0.0.3"));
 
         // Update incoming and outgoing rates:
         state1.incomingRate.update(5); // slow

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java b/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
index 2d12baf..c8469a8 100644
--- a/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
+++ b/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
@@ -18,7 +18,6 @@
 */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
 import java.util.UUID;
 
 import org.junit.BeforeClass;
@@ -32,6 +31,7 @@ import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService.Verb;
 import org.apache.cassandra.schema.MockSchema;
 import org.apache.cassandra.schema.TableMetadata;
@@ -65,7 +65,7 @@ public class WriteCallbackInfoTest
                          ? new Commit(UUID.randomUUID(), new PartitionUpdate.Builder(metadata, ByteBufferUtil.EMPTY_BYTE_BUFFER, RegularAndStaticColumns.NONE, 1).build())
                          : new Mutation(PartitionUpdate.simpleBuilder(metadata, "").build());
 
-        WriteCallbackInfo wcbi = new WriteCallbackInfo(InetAddress.getByName("192.168.1.1"), null, new MessageOut(verb, payload, null), null, cl, allowHints);
+        WriteCallbackInfo wcbi = new WriteCallbackInfo(InetAddressAndPort.getByName("192.168.1.1"), null, new MessageOut(verb, payload, null), null, cl, allowHints);
         Assert.assertEquals(expectHint, wcbi.shouldHint());
         if (expectHint)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index 128fe4b..0211512 100644
--- a/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java
+++ b/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java
@@ -24,6 +24,7 @@ import java.util.Optional;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 
+import com.google.common.net.InetAddresses;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -37,6 +38,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.ChannelWriter.CoalescingChannelWriter;
@@ -68,8 +70,8 @@ public class ChannelWriterTest
     @Before
     public void setup()
     {
-        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.small(new InetSocketAddress("127.0.0.1", 0),
-                                                                             new InetSocketAddress("127.0.0.2", 0));
+        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.small(InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 0),
+                                                                             InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0));
         channel = new EmbeddedChannel();
         omc = new NonSendingOutboundMessagingConnection(id, null, Optional.empty());
         channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 100e1e0..f92ce5a 100644
--- a/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
+++ b/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java
@@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.Optional;
 
+import com.google.common.net.InetAddresses;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -38,6 +39,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -50,8 +52,8 @@ 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 InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9999);
+    private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("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);
 
@@ -179,7 +181,7 @@ public class HandshakeHandlersTest
 
         InboundHandshakeHandler handler = new InboundHandshakeHandler(new TestAuthenticator(true));
         EmbeddedChannel inboundChannel = new EmbeddedChannel(handler);
-        handler.setupMessagingPipeline(inboundChannel.pipeline(), REMOTE_ADDR.getAddress(), compress, MESSAGING_VERSION);
+        handler.setupMessagingPipeline(inboundChannel.pipeline(), REMOTE_ADDR, compress, MESSAGING_VERSION);
 
         return new TestChannels(outboundChannel, inboundChannel);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index a3d646d..af48636 100644
--- a/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java
+++ b/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java
@@ -85,7 +85,7 @@ public class HandshakeProtocolTest
     @Test
     public void thirdMessageTest() throws Exception
     {
-        ThirdHandshakeMessage before = new ThirdHandshakeMessage(MessagingService.current_version, FBUtilities.getBroadcastAddress());
+        ThirdHandshakeMessage before = new ThirdHandshakeMessage(MessagingService.current_version, FBUtilities.getBroadcastAddressAndPort());
         buf = before.encode(PooledByteBufAllocator.DEFAULT);
         ThirdHandshakeMessage after = ThirdHandshakeMessage.maybeDecode(buf);
         assertEquals(before, after);


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


[08/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 48e1b2f..29b45b4 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -37,7 +37,8 @@ public interface StorageServiceMBean extends NotificationEmitter
      *
      * @return set of IP addresses, as Strings
      */
-    public List<String> getLiveNodes();
+    @Deprecated public List<String> getLiveNodes();
+    public List<String> getLiveNodesWithPort();
 
     /**
      * Retrieve the list of unreachable nodes in the cluster, as determined
@@ -45,28 +46,32 @@ public interface StorageServiceMBean extends NotificationEmitter
      *
      * @return set of IP addresses, as Strings
      */
-    public List<String> getUnreachableNodes();
+    @Deprecated public List<String> getUnreachableNodes();
+    public List<String> getUnreachableNodesWithPort();
 
     /**
      * Retrieve the list of nodes currently bootstrapping into the ring.
      *
      * @return set of IP addresses, as Strings
      */
-    public List<String> getJoiningNodes();
+    @Deprecated public List<String> getJoiningNodes();
+    public List<String> getJoiningNodesWithPort();
 
     /**
      * Retrieve the list of nodes currently leaving the ring.
      *
      * @return set of IP addresses, as Strings
      */
-    public List<String> getLeavingNodes();
+    @Deprecated public List<String> getLeavingNodes();
+    public List<String> getLeavingNodesWithPort();
 
     /**
      * Retrieve the list of nodes currently moving in the ring.
      *
      * @return set of IP addresses, as Strings
      */
-    public List<String> getMovingNodes();
+    @Deprecated public List<String> getMovingNodes();
+    public List<String> getMovingNodesWithPort();
 
     /**
      * Fetch string representations of the tokens for this node.
@@ -120,7 +125,8 @@ public interface StorageServiceMBean extends NotificationEmitter
      *
      * @return mapping of ranges to end points
      */
-    public Map<List<String>, List<String>> getRangeToEndpointMap(String keyspace);
+    @Deprecated public Map<List<String>, List<String>> getRangeToEndpointMap(String keyspace);
+    public Map<List<String>, List<String>> getRangeToEndpointWithPortMap(String keyspace);
 
     /**
      * Retrieve a map of range to rpc addresses that describe the ring topology
@@ -128,7 +134,8 @@ public interface StorageServiceMBean extends NotificationEmitter
      *
      * @return mapping of ranges to rpc addresses
      */
-    public Map<List<String>, List<String>> getRangeToRpcaddressMap(String keyspace);
+    @Deprecated public Map<List<String>, List<String>> getRangeToRpcaddressMap(String keyspace);
+    public Map<List<String>, List<String>> getRangeToNativeaddressWithPortMap(String keyspace);
 
     /**
      * The same as {@code describeRing(String)} but converts TokenRange to the String for JMX compatibility
@@ -137,14 +144,16 @@ public interface StorageServiceMBean extends NotificationEmitter
      *
      * @return a List of TokenRange(s) converted to String for the given keyspace
      */
-    public List <String> describeRingJMX(String keyspace) throws IOException;
+    @Deprecated public List <String> describeRingJMX(String keyspace) throws IOException;
+    public List<String> describeRingWithPortJMX(String keyspace) throws IOException;
 
     /**
      * Retrieve a map of pending ranges to endpoints that describe the ring topology
      * @param keyspace the keyspace to get the pending range map for.
      * @return a map of pending ranges to endpoints
      */
-    public Map<List<String>, List<String>> getPendingRangeToEndpointMap(String keyspace);
+    @Deprecated public Map<List<String>, List<String>> getPendingRangeToEndpointMap(String keyspace);
+    public Map<List<String>, List<String>> getPendingRangeToEndpointWithPortMap(String keyspace);
 
     /**
      * Retrieve a map of tokens to endpoints, including the bootstrapping
@@ -152,7 +161,8 @@ public interface StorageServiceMBean extends NotificationEmitter
      *
      * @return a map of tokens to endpoints in ascending order
      */
-    public Map<String, String> getTokenToEndpointMap();
+    @Deprecated public Map<String, String> getTokenToEndpointMap();
+    public Map<String, String> getTokenToEndpointWithPortMap();
 
     /** Retrieve this hosts unique ID */
     public String getLocalHostId();
@@ -162,16 +172,19 @@ public interface StorageServiceMBean extends NotificationEmitter
     public Map<String, String> getHostIdMap();
 
     /** Retrieve the mapping of endpoint to host ID */
-    public Map<String, String> getEndpointToHostId();
+    @Deprecated public Map<String, String> getEndpointToHostId();
+    public Map<String, String> getEndpointWithPortToHostId();
 
     /** Retrieve the mapping of host ID to endpoint */
-    public Map<String, String> getHostIdToEndpoint();
+    @Deprecated public Map<String, String> getHostIdToEndpoint();
+    public Map<String, String> getHostIdToEndpointWithPort();
 
     /** Human-readable load value */
     public String getLoadString();
 
     /** Human-readable load value.  Keys are IP addresses. */
-    public Map<String, String> getLoadMap();
+    @Deprecated public Map<String, String> getLoadMap();
+    public Map<String, String> getLoadMapWithPort();
 
     /**
      * Return the generation value for this node.
@@ -189,8 +202,10 @@ public interface StorageServiceMBean extends NotificationEmitter
      * @param key - key for which we need to find the endpoint return value -
      * the endpoint responsible for this key
      */
-    public List<InetAddress> getNaturalEndpoints(String keyspaceName, String cf, String key);
-    public List<InetAddress> getNaturalEndpoints(String keyspaceName, ByteBuffer key);
+    @Deprecated public List<InetAddress> getNaturalEndpoints(String keyspaceName, String cf, String key);
+    public List<String> getNaturalEndpointsWithPort(String keyspaceName, String cf, String key);
+    @Deprecated public List<InetAddress> getNaturalEndpoints(String keyspaceName, ByteBuffer key);
+    public List<String> getNaturalEndpointsWithPort(String keysapceName, ByteBuffer key);
 
     /**
      * @deprecated use {@link #takeSnapshot(String tag, Map options, String... entities)} instead.
@@ -353,7 +368,8 @@ public interface StorageServiceMBean extends NotificationEmitter
     /**
      * Get the status of a token removal.
      */
-    public String getRemovalStatus();
+    @Deprecated public String getRemovalStatus();
+    public String getRemovalStatusWithPort();
 
     /**
      * Force a remove operation to finish.
@@ -408,7 +424,8 @@ public interface StorageServiceMBean extends NotificationEmitter
      * given a list of tokens (representing the nodes in the cluster), returns
      *   a mapping from {@code "token -> %age of cluster owned by that token"}
      */
-    public Map<InetAddress, Float> getOwnership();
+    @Deprecated public Map<InetAddress, Float> getOwnership();
+    public Map<String, Float> getOwnershipWithPort();
 
     /**
      * Effective ownership is % of the data each node owns given the keyspace
@@ -417,7 +434,8 @@ public interface StorageServiceMBean extends NotificationEmitter
      * in the cluster have the same replication strategies and if yes then we will
      * use the first else a empty Map is returned.
      */
-    public Map<InetAddress, Float> effectiveOwnership(String keyspace) throws IllegalStateException;
+    @Deprecated public Map<InetAddress, Float> effectiveOwnership(String keyspace) throws IllegalStateException;
+    public Map<String, Float> effectiveOwnershipWithPort(String keyspace) throws IllegalStateException;
 
     public List<String> getKeyspaces();
 
@@ -425,7 +443,8 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     public List<String> getNonLocalStrategyKeyspaces();
 
-    public Map<String, String> getViewBuildStatuses(String keyspace, String view);
+    @Deprecated public Map<String, String> getViewBuildStatuses(String keyspace, String view);
+    public Map<String, String> getViewBuildStatusesWithPort(String keyspace, String view);
 
     /**
      * Change endpointsnitch class and dynamic-ness (and dynamic attributes) at runtime.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/TokenRange.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/TokenRange.java b/src/java/org/apache/cassandra/service/TokenRange.java
index 0e46910..a1f9aee 100644
--- a/src/java/org/apache/cassandra/service/TokenRange.java
+++ b/src/java/org/apache/cassandra/service/TokenRange.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.*;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Holds token range informations for the sake of {@link StorageService#describeRing}.
@@ -54,13 +54,13 @@ public class TokenRange
         return tokenFactory.toString(tk);
     }
 
-    public static TokenRange create(Token.TokenFactory tokenFactory, Range<Token> range, List<InetAddress> endpoints)
+    public static TokenRange create(Token.TokenFactory tokenFactory, Range<Token> range, List<InetAddressAndPort> endpoints, boolean withPorts)
     {
         List<EndpointDetails> details = new ArrayList<>(endpoints.size());
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        for (InetAddress ep : endpoints)
+        for (InetAddressAndPort ep : endpoints)
             details.add(new EndpointDetails(ep,
-                                            StorageService.instance.getRpcaddress(ep),
+                                            StorageService.instance.getNativeaddress(ep, withPorts),
                                             snitch.getDatacenter(ep),
                                             snitch.getRack(ep)));
         return new TokenRange(tokenFactory, range, details);
@@ -69,6 +69,11 @@ public class TokenRange
     @Override
     public String toString()
     {
+        return toString(false);
+    }
+
+    public String toString(boolean withPorts)
+    {
         StringBuilder sb = new StringBuilder("TokenRange(");
 
         sb.append("start_token:").append(toStr(range.left));
@@ -76,33 +81,43 @@ public class TokenRange
 
         List<String> hosts = new ArrayList<>(endpoints.size());
         List<String> rpcs = new ArrayList<>(endpoints.size());
+        List<String> endpointDetails = new ArrayList<>(endpoints.size());
         for (EndpointDetails ep : endpoints)
         {
-            hosts.add(ep.host.getHostAddress());
-            rpcs.add(ep.rpcAddress);
+            hosts.add(ep.host.getHostAddress(withPorts));
+            rpcs.add(ep.nativeAddress);
+            endpointDetails.add(ep.toString(withPorts));
         }
 
-        sb.append("endpoints:").append(hosts);
-        sb.append("rpc_endpoints:").append(rpcs);
-        sb.append("endpoint_details:").append(endpoints);
-
+        if (withPorts)
+        {
+            sb.append(", endpoints:").append(hosts);
+            sb.append(", rpc_endpoints:").append(rpcs);
+            sb.append(", endpoint_details:").append(endpointDetails);
+        }
+        else
+        {
+            sb.append("endpoints:").append(hosts);
+            sb.append("rpc_endpoints:").append(rpcs);
+            sb.append("endpoint_details:").append(endpointDetails);
+        }
         sb.append(")");
         return sb.toString();
     }
 
     public static class EndpointDetails
     {
-        public final InetAddress host;
-        public final String rpcAddress;
+        public final InetAddressAndPort host;
+        public final String nativeAddress;
         public final String datacenter;
         public final String rack;
 
-        private EndpointDetails(InetAddress host, String rpcAddress, String datacenter, String rack)
+        private EndpointDetails(InetAddressAndPort host, String nativeAddress, String datacenter, String rack)
         {
             // dc and rack can be null, but host shouldn't
             assert host != null;
             this.host = host;
-            this.rpcAddress = rpcAddress;
+            this.nativeAddress = nativeAddress;
             this.datacenter = datacenter;
             this.rack = rack;
         }
@@ -110,10 +125,15 @@ public class TokenRange
         @Override
         public String toString()
         {
+            return toString(false);
+        }
+
+        public String toString(boolean withPorts)
+        {
             // Format matters for backward compatibility with describeRing()
             String dcStr = datacenter == null ? "" : String.format(", datacenter:%s", datacenter);
             String rackStr = rack == null ? "" : String.format(", rack:%s", rack);
-            return String.format("EndpointDetails(host:%s%s%s)", host.getHostAddress(), dcStr, rackStr);
+            return String.format("EndpointDetails(host:%s%s%s)", host.getHostAddress(withPorts), dcStr, rackStr);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/WriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/WriteResponseHandler.java b/src/java/org/apache/cassandra/service/WriteResponseHandler.java
index 55ca5aa..65efeff 100644
--- a/src/java/org/apache/cassandra/service/WriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/WriteResponseHandler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -27,6 +26,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
@@ -42,8 +42,8 @@ public class WriteResponseHandler<T> extends AbstractWriteResponseHandler<T>
     private static final AtomicIntegerFieldUpdater<WriteResponseHandler> responsesUpdater
             = AtomicIntegerFieldUpdater.newUpdater(WriteResponseHandler.class, "responses");
 
-    public WriteResponseHandler(Collection<InetAddress> writeEndpoints,
-                                Collection<InetAddress> pendingEndpoints,
+    public WriteResponseHandler(Collection<InetAddressAndPort> writeEndpoints,
+                                Collection<InetAddressAndPort> pendingEndpoints,
                                 ConsistencyLevel consistencyLevel,
                                 Keyspace keyspace,
                                 Runnable callback,
@@ -54,12 +54,12 @@ public class WriteResponseHandler<T> extends AbstractWriteResponseHandler<T>
         responses = totalBlockFor();
     }
 
-    public WriteResponseHandler(InetAddress endpoint, WriteType writeType, Runnable callback, long queryStartNanoTime)
+    public WriteResponseHandler(InetAddressAndPort endpoint, WriteType writeType, Runnable callback, long queryStartNanoTime)
     {
-        this(Arrays.asList(endpoint), Collections.<InetAddress>emptyList(), ConsistencyLevel.ONE, null, callback, writeType, queryStartNanoTime);
+        this(Arrays.asList(endpoint), Collections.<InetAddressAndPort>emptyList(), ConsistencyLevel.ONE, null, callback, writeType, queryStartNanoTime);
     }
 
-    public WriteResponseHandler(InetAddress endpoint, WriteType writeType, long queryStartNanoTime)
+    public WriteResponseHandler(InetAddressAndPort endpoint, WriteType writeType, long queryStartNanoTime)
     {
         this(endpoint, writeType, null, queryStartNanoTime);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
index 381c498..ed70e96 100644
--- a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.service.paxos;
  */
 
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -29,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.DecoratedKey;
@@ -48,7 +48,7 @@ public class PrepareCallback extends AbstractPaxosCallback<PrepareResponse>
     public Commit mostRecentInProgressCommit;
     public Commit mostRecentInProgressCommitWithUpdate;
 
-    private final Map<InetAddress, Commit> commitsByReplica = new ConcurrentHashMap<InetAddress, Commit>();
+    private final Map<InetAddressAndPort, Commit> commitsByReplica = new ConcurrentHashMap<>();
 
     public PrepareCallback(DecoratedKey key, TableMetadata metadata, int targets, ConsistencyLevel consistency, long queryStartNanoTime)
     {
@@ -90,7 +90,7 @@ public class PrepareCallback extends AbstractPaxosCallback<PrepareResponse>
         latch.countDown();
     }
 
-    public Iterable<InetAddress> replicasMissingMostRecentCommit(TableMetadata metadata, int nowInSec)
+    public Iterable<InetAddressAndPort> replicasMissingMostRecentCommit(TableMetadata metadata, int nowInSec)
     {
         // In general, we need every replicas that have answered to the prepare (a quorum) to agree on the MRC (see
         // coment in StorageProxy.beginAndRepairPaxos(), but basically we need to make sure at least a quorum of nodes
@@ -105,9 +105,9 @@ public class PrepareCallback extends AbstractPaxosCallback<PrepareResponse>
         if (UUIDGen.unixTimestampInSec(mostRecentCommit.ballot) + paxosTtlSec < nowInSec)
             return Collections.emptySet();
 
-        return Iterables.filter(commitsByReplica.keySet(), new Predicate<InetAddress>()
+        return Iterables.filter(commitsByReplica.keySet(), new Predicate<InetAddressAndPort>()
         {
-            public boolean apply(InetAddress inetAddress)
+            public boolean apply(InetAddressAndPort inetAddress)
             {
                 return (!commitsByReplica.get(inetAddress).ballot.equals(mostRecentCommit.ballot));
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/streaming/ProgressInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/ProgressInfo.java b/src/java/org/apache/cassandra/streaming/ProgressInfo.java
index fdd3e97..2334599 100644
--- a/src/java/org/apache/cassandra/streaming/ProgressInfo.java
+++ b/src/java/org/apache/cassandra/streaming/ProgressInfo.java
@@ -18,10 +18,11 @@
 package org.apache.cassandra.streaming;
 
 import java.io.Serializable;
-import java.net.InetAddress;
 
 import com.google.common.base.Objects;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 /**
  * ProgressInfo contains file transfer progress.
  */
@@ -48,14 +49,14 @@ public class ProgressInfo implements Serializable
         }
     }
 
-    public final InetAddress peer;
+    public final InetAddressAndPort peer;
     public final int sessionIndex;
     public final String fileName;
     public final Direction direction;
     public final long currentBytes;
     public final long totalBytes;
 
-    public ProgressInfo(InetAddress peer, int sessionIndex, String fileName, Direction direction, long currentBytes, long totalBytes)
+    public ProgressInfo(InetAddressAndPort peer, int sessionIndex, String fileName, Direction direction, long currentBytes, long totalBytes)
     {
         assert totalBytes > 0;
 
@@ -102,13 +103,18 @@ public class ProgressInfo implements Serializable
     @Override
     public String toString()
     {
+        return toString(false);
+    }
+
+    public String toString(boolean withPorts)
+    {
         StringBuilder sb = new StringBuilder(fileName);
         sb.append(" ").append(currentBytes);
         sb.append("/").append(totalBytes).append(" bytes");
         sb.append("(").append(currentBytes*100/totalBytes).append("%) ");
         sb.append(direction == Direction.OUT ? "sent to " : "received from ");
         sb.append("idx:").append(sessionIndex);
-        sb.append(peer);
+        sb.append(peer.toString(withPorts));
         return sb.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/streaming/SessionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/SessionInfo.java b/src/java/org/apache/cassandra/streaming/SessionInfo.java
index 1521614..bbca753 100644
--- a/src/java/org/apache/cassandra/streaming/SessionInfo.java
+++ b/src/java/org/apache/cassandra/streaming/SessionInfo.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.streaming;
 
 import java.io.Serializable;
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -27,6 +26,7 @@ import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -34,9 +34,9 @@ import org.apache.cassandra.utils.FBUtilities;
  */
 public final class SessionInfo implements Serializable
 {
-    public final InetAddress peer;
+    public final InetAddressAndPort peer;
     public final int sessionIndex;
-    public final InetAddress connecting;
+    public final InetAddressAndPort connecting;
     /** Immutable collection of receiving summaries */
     public final Collection<StreamSummary> receivingSummaries;
     /** Immutable collection of sending summaries*/
@@ -47,9 +47,9 @@ public final class SessionInfo implements Serializable
     private final Map<String, ProgressInfo> receivingFiles;
     private final Map<String, ProgressInfo> sendingFiles;
 
-    public SessionInfo(InetAddress peer,
+    public SessionInfo(InetAddressAndPort peer,
                        int sessionIndex,
-                       InetAddress connecting,
+                       InetAddressAndPort connecting,
                        Collection<StreamSummary> receivingSummaries,
                        Collection<StreamSummary> sendingSummaries,
                        StreamSession.State state)
@@ -195,6 +195,6 @@ public final class SessionInfo implements Serializable
 
     public SessionSummary createSummary()
     {
-        return new SessionSummary(FBUtilities.getBroadcastAddress(), peer, receivingSummaries, sendingSummaries);
+        return new SessionSummary(FBUtilities.getBroadcastAddressAndPort(), peer, receivingSummaries, sendingSummaries);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/streaming/SessionSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/SessionSummary.java b/src/java/org/apache/cassandra/streaming/SessionSummary.java
index d52c2ca..cf63a57 100644
--- a/src/java/org/apache/cassandra/streaming/SessionSummary.java
+++ b/src/java/org/apache/cassandra/streaming/SessionSummary.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.streaming;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -28,19 +27,19 @@ 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.DataOutputPlus;
-import org.apache.cassandra.serializers.InetAddressSerializer;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 
 public class SessionSummary
 {
-    public final InetAddress coordinator;
-    public final InetAddress peer;
+    public final InetAddressAndPort coordinator;
+    public final InetAddressAndPort peer;
     /** Immutable collection of receiving summaries */
     public final Collection<StreamSummary> receivingSummaries;
     /** Immutable collection of sending summaries*/
     public final Collection<StreamSummary> sendingSummaries;
 
-    public SessionSummary(InetAddress coordinator, InetAddress peer,
+    public SessionSummary(InetAddressAndPort coordinator, InetAddressAndPort peer,
                           Collection<StreamSummary> receivingSummaries,
                           Collection<StreamSummary> sendingSummaries)
     {
@@ -81,8 +80,8 @@ public class SessionSummary
     {
         public void serialize(SessionSummary summary, DataOutputPlus out, int version) throws IOException
         {
-            ByteBufferUtil.writeWithLength(InetAddressSerializer.instance.serialize(summary.coordinator), out);
-            ByteBufferUtil.writeWithLength(InetAddressSerializer.instance.serialize(summary.peer), out);
+            CompactEndpointSerializationHelper.instance.serialize(summary.coordinator, out, version);
+            CompactEndpointSerializationHelper.instance.serialize(summary.peer, out, version);
 
             out.writeInt(summary.receivingSummaries.size());
             for (StreamSummary streamSummary: summary.receivingSummaries)
@@ -99,8 +98,8 @@ public class SessionSummary
 
         public SessionSummary deserialize(DataInputPlus in, int version) throws IOException
         {
-            InetAddress coordinator = InetAddressSerializer.instance.deserialize(ByteBufferUtil.readWithLength(in));
-            InetAddress peer = InetAddressSerializer.instance.deserialize(ByteBufferUtil.readWithLength(in));
+            InetAddressAndPort coordinator = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+            InetAddressAndPort peer = CompactEndpointSerializationHelper.instance.deserialize(in, version);
 
             int numRcvd = in.readInt();
             List<StreamSummary> receivingSummaries = new ArrayList<>(numRcvd);
@@ -122,8 +121,8 @@ public class SessionSummary
         public long serializedSize(SessionSummary summary, int version)
         {
             long size = 0;
-            size += ByteBufferUtil.serializedSizeWithLength(InetAddressSerializer.instance.serialize(summary.coordinator));
-            size += ByteBufferUtil.serializedSizeWithLength(InetAddressSerializer.instance.serialize(summary.peer));
+            size += CompactEndpointSerializationHelper.instance.serializedSize(summary.coordinator, version);
+            size += CompactEndpointSerializationHelper.instance.serializedSize(summary.peer, version);
 
             size += TypeSizes.sizeof(summary.receivingSummaries.size());
             for (StreamSummary streamSummary: summary.receivingSummaries)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 bb8c702..a22e07d 100644
--- a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
+++ b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.net.InetAddress;
 import java.util.*;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -45,7 +45,7 @@ public class StreamCoordinator
                                                                                                                             FBUtilities.getAvailableProcessors());
     private final boolean connectSequentially;
 
-    private Map<InetAddress, HostStreamingData> peerSessions = new HashMap<>();
+    private Map<InetAddressAndPort, HostStreamingData> peerSessions = new HashMap<>();
     private final int connectionsPerHost;
     private StreamConnectionFactory factory;
     private final boolean keepSSTableLevel;
@@ -143,29 +143,29 @@ public class StreamCoordinator
         if (sessionsToConnect.hasNext())
         {
             StreamSession next = sessionsToConnect.next();
-            logger.debug("Connecting next session {} with {}.", next.planId(), next.peer.getHostAddress());
+            logger.debug("Connecting next session {} with {}.", next.planId(), next.peer.toString());
             streamExecutor.execute(new StreamSessionConnector(next));
         }
         else
             logger.debug("Finished connecting all sessions");
     }
 
-    public synchronized Set<InetAddress> getPeers()
+    public synchronized Set<InetAddressAndPort> getPeers()
     {
         return new HashSet<>(peerSessions.keySet());
     }
 
-    public synchronized StreamSession getOrCreateNextSession(InetAddress peer, InetAddress connecting)
+    public synchronized StreamSession getOrCreateNextSession(InetAddressAndPort peer, InetAddressAndPort connecting)
     {
         return getOrCreateHostData(peer).getOrCreateNextSession(peer, connecting);
     }
 
-    public synchronized StreamSession getOrCreateSessionById(InetAddress peer, int id, InetAddress connecting)
+    public synchronized StreamSession getOrCreateSessionById(InetAddressAndPort peer, int id, InetAddressAndPort connecting)
     {
         return getOrCreateHostData(peer).getOrCreateSessionById(peer, id, connecting);
     }
 
-    public StreamSession getSessionById(InetAddress peer, int id)
+    public StreamSession getSessionById(InetAddressAndPort peer, int id)
     {
         return getHostData(peer).getSessionById(id);
     }
@@ -191,7 +191,7 @@ public class StreamCoordinator
         return result;
     }
 
-    public synchronized void transferFiles(InetAddress to, Collection<StreamSession.SSTableStreamingSections> sstableDetails)
+    public synchronized void transferFiles(InetAddressAndPort to, Collection<StreamSession.SSTableStreamingSections> sstableDetails)
     {
         HostStreamingData sessionList = getOrCreateHostData(to);
 
@@ -239,7 +239,7 @@ public class StreamCoordinator
         return result;
     }
 
-    private HostStreamingData getHostData(InetAddress peer)
+    private HostStreamingData getHostData(InetAddressAndPort peer)
     {
         HostStreamingData data = peerSessions.get(peer);
         if (data == null)
@@ -247,7 +247,7 @@ public class StreamCoordinator
         return data;
     }
 
-    private HostStreamingData getOrCreateHostData(InetAddress peer)
+    private HostStreamingData getOrCreateHostData(InetAddressAndPort peer)
     {
         HostStreamingData data = peerSessions.get(peer);
         if (data == null)
@@ -297,7 +297,7 @@ public class StreamCoordinator
             return false;
         }
 
-        public StreamSession getOrCreateNextSession(InetAddress peer, InetAddress connecting)
+        public StreamSession getOrCreateNextSession(InetAddressAndPort peer, InetAddressAndPort connecting)
         {
             // create
             if (streamSessions.size() < connectionsPerHost)
@@ -329,7 +329,7 @@ public class StreamCoordinator
             return Collections.unmodifiableCollection(streamSessions.values());
         }
 
-        public StreamSession getOrCreateSessionById(InetAddress peer, int id, InetAddress connecting)
+        public StreamSession getOrCreateSessionById(InetAddressAndPort peer, int id, InetAddressAndPort connecting)
         {
             StreamSession session = streamSessions.get(id);
             if (session == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/streaming/StreamEvent.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamEvent.java b/src/java/org/apache/cassandra/streaming/StreamEvent.java
index 6ea2814..7ecd081 100644
--- a/src/java/org/apache/cassandra/streaming/StreamEvent.java
+++ b/src/java/org/apache/cassandra/streaming/StreamEvent.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
@@ -27,6 +26,7 @@ import com.google.common.collect.ImmutableSet;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public abstract class StreamEvent
 {
@@ -48,7 +48,7 @@ public abstract class StreamEvent
 
     public static class SessionCompleteEvent extends StreamEvent
     {
-        public final InetAddress peer;
+        public final InetAddressAndPort peer;
         public final boolean success;
         public final int sessionIndex;
         public final Set<StreamRequest> requests;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 a44f02e..81c65c5 100644
--- a/src/java/org/apache/cassandra/streaming/StreamManager.java
+++ b/src/java/org/apache/cassandra/streaming/StreamManager.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.net.InetAddress;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -35,6 +34,7 @@ import com.google.common.util.concurrent.RateLimiter;
 
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.management.StreamEventJMXNotifier;
 import org.apache.cassandra.streaming.management.StreamStateCompositeData;
 
@@ -55,7 +55,7 @@ public class StreamManager implements StreamManagerMBean
      *
      * @return StreamRateLimiter with rate limit set based on peer location.
      */
-    public static StreamRateLimiter getRateLimiter(InetAddress peer)
+    public static StreamRateLimiter getRateLimiter(InetAddressAndPort peer)
     {
         return new StreamRateLimiter(peer);
     }
@@ -67,7 +67,7 @@ public class StreamManager implements StreamManagerMBean
         private static final RateLimiter interDCLimiter = RateLimiter.create(Double.MAX_VALUE);
         private final boolean isLocalDC;
 
-        public StreamRateLimiter(InetAddress peer)
+        public StreamRateLimiter(InetAddressAndPort peer)
         {
             double throughput = DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec() * BYTES_PER_MEGABIT;
             mayUpdateThroughput(throughput, limiter);
@@ -176,7 +176,7 @@ public class StreamManager implements StreamManagerMBean
         return notifier.getNotificationInfo();
     }
 
-    public StreamSession findSession(InetAddress peer, UUID planId, int sessionIndex)
+    public StreamSession findSession(InetAddressAndPort peer, UUID planId, int sessionIndex)
     {
         StreamSession session = findSession(initiatedStreams, peer, planId, sessionIndex);
         if (session !=  null)
@@ -185,7 +185,7 @@ public class StreamManager implements StreamManagerMBean
         return findSession(receivingStreams, peer, planId, sessionIndex);
     }
 
-    private StreamSession findSession(Map<UUID, StreamResultFuture> streams, InetAddress peer, UUID planId, int sessionIndex)
+    private StreamSession findSession(Map<UUID, StreamResultFuture> streams, InetAddressAndPort peer, UUID planId, int sessionIndex)
     {
         StreamResultFuture streamResultFuture = streams.get(planId);
         if (streamResultFuture == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 213f74b..43e9068 100644
--- a/src/java/org/apache/cassandra/streaming/StreamPlan.java
+++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java
@@ -17,11 +17,11 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.net.InetAddress;
 import java.util.*;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.UUIDGen;
 
 import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
@@ -73,7 +73,7 @@ public class StreamPlan
      * @param ranges ranges to fetch
      * @return this object for chaining
      */
-    public StreamPlan requestRanges(InetAddress from, InetAddress connecting, String keyspace, Collection<Range<Token>> ranges)
+    public StreamPlan requestRanges(InetAddressAndPort from, InetAddressAndPort connecting, String keyspace, Collection<Range<Token>> ranges)
     {
         return requestRanges(from, connecting, keyspace, ranges, EMPTY_COLUMN_FAMILIES);
     }
@@ -88,7 +88,7 @@ public class StreamPlan
      * @param columnFamilies specific column families
      * @return this object for chaining
      */
-    public StreamPlan requestRanges(InetAddress from, InetAddress connecting, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+    public StreamPlan requestRanges(InetAddressAndPort from, InetAddressAndPort connecting, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
     {
         StreamSession session = coordinator.getOrCreateNextSession(from, connecting);
         session.addStreamRequest(keyspace, ranges, Arrays.asList(columnFamilies));
@@ -98,9 +98,9 @@ public class StreamPlan
     /**
      * Add transfer task to send data of specific {@code columnFamilies} under {@code keyspace} and {@code ranges}.
      *
-     * @see #transferRanges(java.net.InetAddress, java.net.InetAddress, String, java.util.Collection, String...)
+     * @see #transferRanges(InetAddressAndPort, InetAddressAndPort, String, java.util.Collection, String...)
      */
-    public StreamPlan transferRanges(InetAddress to, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+    public StreamPlan transferRanges(InetAddressAndPort to, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
     {
         return transferRanges(to, to, keyspace, ranges, columnFamilies);
     }
@@ -114,7 +114,7 @@ public class StreamPlan
      * @param ranges ranges to send
      * @return this object for chaining
      */
-    public StreamPlan transferRanges(InetAddress to, InetAddress connecting, String keyspace, Collection<Range<Token>> ranges)
+    public StreamPlan transferRanges(InetAddressAndPort to, InetAddressAndPort connecting, String keyspace, Collection<Range<Token>> ranges)
     {
         return transferRanges(to, connecting, keyspace, ranges, EMPTY_COLUMN_FAMILIES);
     }
@@ -129,7 +129,7 @@ public class StreamPlan
      * @param columnFamilies specific column families
      * @return this object for chaining
      */
-    public StreamPlan transferRanges(InetAddress to, InetAddress connecting, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+    public StreamPlan transferRanges(InetAddressAndPort to, InetAddressAndPort connecting, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
     {
         StreamSession session = coordinator.getOrCreateNextSession(to, connecting);
         session.addTransferRanges(keyspace, ranges, Arrays.asList(columnFamilies), flushBeforeTransfer);
@@ -144,7 +144,7 @@ public class StreamPlan
      *                       this collection will be modified to remove those files that are successfully handed off
      * @return this object for chaining
      */
-    public StreamPlan transferFiles(InetAddress to, Collection<StreamSession.SSTableStreamingSections> sstableDetails)
+    public StreamPlan transferFiles(InetAddressAndPort to, Collection<StreamSession.SSTableStreamingSections> sstableDetails)
     {
         coordinator.transferFiles(to, sstableDetails);
         return this;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 0f74c7f..544f37f 100644
--- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import io.netty.channel.Channel;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -103,7 +104,7 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
     public static synchronized StreamResultFuture initReceivingSide(int sessionIndex,
                                                                     UUID planId,
                                                                     StreamOperation streamOperation,
-                                                                    InetAddress from,
+                                                                    InetAddressAndPort from,
                                                                     Channel channel,
                                                                     boolean keepSSTableLevel,
                                                                     UUID pendingRepair,
@@ -135,11 +136,15 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
         return coordinator;
     }
 
-    private void attachConnection(InetAddress from, int sessionIndex, Channel channel)
+    private void attachConnection(InetAddressAndPort 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);
+        //In the case of unit tests, if you use the EmbeddedChannel, channel.remoteAddress()
+        //does not return an InetSocketAddress, but an EmbeddedSocketAddress. Hence why we need the type check here
+        InetAddress connecting = (addr instanceof InetSocketAddress ? ((InetSocketAddress) addr).getAddress() : from.address);
+        //Need to turn connecting into a InetAddressAndPort with the correct port. I think getting the port from "from"
+        //Will work since we don't actually have ports diverge across network interfaces
+        StreamSession session = coordinator.getOrCreateSessionById(from, sessionIndex, InetAddressAndPort.getByAddressOverrideDefaults(connecting, from.port));
         session.init(this);
         session.attach(channel);
     }
@@ -228,7 +233,7 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
         }
     }
 
-    StreamSession getSession(InetAddress peer, int sessionIndex)
+    StreamSession getSession(InetAddressAndPort peer, int sessionIndex)
     {
         return coordinator.getSessionById(peer, sessionIndex);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 b6351f9..4085c43 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -49,6 +49,7 @@ import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.*;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.StreamingMetrics;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
@@ -142,14 +143,14 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     /**
      * Streaming endpoint.
      *
-     * Each {@code StreamSession} is identified by this InetAddress which is broadcast address of the node streaming.
+     * Each {@code StreamSession} is identified by this InetAddressAndPort which is broadcast address of the node streaming.
      */
-    public final InetAddress peer;
+    public final InetAddressAndPort peer;
 
     private final int index;
 
     /** Actual connecting address. Can be the same as {@linkplain #peer}. */
-    public final InetAddress connecting;
+    public final InetAddressAndPort connecting;
 
     // should not be null when session is started
     private StreamResultFuture streamResult;
@@ -191,14 +192,14 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      *  @param peer Address of streaming peer
      * @param connecting Actual connecting address
      */
-    public StreamSession(InetAddress peer, InetAddress connecting, StreamConnectionFactory factory, int index, boolean keepSSTableLevel, UUID pendingRepair, PreviewKind previewKind)
+    public StreamSession(InetAddressAndPort peer, InetAddressAndPort connecting, StreamConnectionFactory factory, int index, boolean keepSSTableLevel, UUID pendingRepair, PreviewKind previewKind)
     {
         this.peer = peer;
         this.connecting = connecting;
         this.index = index;
 
-        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.stream(new InetSocketAddress(FBUtilities.getBroadcastAddress(), 0),
-                                                                              new InetSocketAddress(connecting, MessagingService.instance().portFor(connecting)));
+        OutboundConnectionIdentifier id = OutboundConnectionIdentifier.stream(InetAddressAndPort.getByAddressOverrideDefaults(FBUtilities.getBroadcastAddressAndPort().address, 0),
+                                                                              InetAddressAndPort.getByAddressOverrideDefaults(connecting.address, MessagingService.instance().portFor(connecting)));
         this.messageSender = new NettyStreamingMessageSender(this, id, factory, StreamMessage.CURRENT_VERSION, previewKind.isPreview());
         this.metrics = StreamingMetrics.get(connecting);
         this.keepSSTableLevel = keepSSTableLevel;
@@ -607,16 +608,16 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         {
             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(),
+                         peer.getHostAddress(true),
+                         peer.equals(connecting) ? "" : " through " + connecting.getHostAddress(true),
                          2 * DatabaseDescriptor.getStreamingKeepAlivePeriod(),
                          e);
         }
         else
         {
             logger.error("[Stream #{}] Streaming error occurred on session with peer {}{}", planId(),
-                         peer.getHostAddress(),
-                         peer.equals(connecting) ? "" : " through " + connecting.getHostAddress(),
+                         peer.getHostAddress(true),
+                         peer.equals(connecting) ? "" : " through " + connecting.getHostAddress(true),
                          e);
         }
     }
@@ -644,7 +645,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
             prepareReceiving(summary);
 
         PrepareSynAckMessage prepareSynAck = new PrepareSynAckMessage();
-        if (!peer.equals(FBUtilities.getBroadcastAddress()))
+        if (!peer.equals(FBUtilities.getBroadcastAddressAndPort()))
             for (StreamTransferTask task : transfers.values())
                 prepareSynAck.summaries.add(task.getSummary());
         messageSender.sendMessage(prepareSynAck);
@@ -754,7 +755,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      */
     public synchronized void sessionFailed()
     {
-        logger.error("[Stream #{}] Remote peer {} failed stream session.", planId(), peer.getHostAddress());
+        logger.error("[Stream #{}] Remote peer {} failed stream session.", planId(), peer.toString());
         closeSession(State.FAILED);
     }
 
@@ -784,21 +785,21 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         maybeCompleted();
     }
 
-    public void onJoin(InetAddress endpoint, EndpointState epState) {}
-    public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
-    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) {}
-    public void onAlive(InetAddress endpoint, EndpointState state) {}
-    public void onDead(InetAddress endpoint, EndpointState state) {}
+    public void onJoin(InetAddressAndPort endpoint, EndpointState epState) {}
+    public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
+    public void onChange(InetAddressAndPort endpoint, ApplicationState state, VersionedValue value) {}
+    public void onAlive(InetAddressAndPort endpoint, EndpointState state) {}
+    public void onDead(InetAddressAndPort endpoint, EndpointState state) {}
 
-    public void onRemove(InetAddress endpoint)
+    public void onRemove(InetAddressAndPort endpoint)
     {
-        logger.error("[Stream #{}] Session failed because remote peer {} has left.", planId(), peer.getHostAddress());
+        logger.error("[Stream #{}] Session failed because remote peer {} has left.", planId(), peer.toString());
         closeSession(State.FAILED);
     }
 
-    public void onRestart(InetAddress endpoint, EndpointState epState)
+    public void onRestart(InetAddressAndPort endpoint, EndpointState epState)
     {
-        logger.error("[Stream #{}] Session failed because remote peer {} was restarted.", planId(), peer.getHostAddress());
+        logger.error("[Stream #{}] Session failed because remote peer {} was restarted.", planId(), peer.toString());
         closeSession(State.FAILED);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index 0b38760..20b7c87 100644
--- a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
+++ b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
@@ -137,7 +137,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
     @Override
     public void initialize() throws IOException
     {
-        StreamInitMessage message = new StreamInitMessage(FBUtilities.getBroadcastAddress(),
+        StreamInitMessage message = new StreamInitMessage(FBUtilities.getBroadcastAddressAndPort(),
                                                           session.sessionIndex(),
                                                           session.planId(),
                                                           session.streamOperation(),
@@ -183,7 +183,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
     {
         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));
+        pipeline.addLast(NettyFactory.instance.streamingGroup, NettyFactory.INBOUND_STREAM_HANDLER_NAME, new StreamingInboundHandler(connectionId.remote(), protocolVersion, session));
         channel.attr(TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
         return channel;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index cc6f9e0..907572b 100644
--- a/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java
+++ b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java
@@ -21,7 +21,6 @@ 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;
@@ -38,6 +37,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.streaming.StreamReceiveException;
@@ -65,7 +65,7 @@ public class StreamingInboundHandler extends ChannelInboundHandlerAdapter
     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 InetAddressAndPort remoteAddress;
     private final int protocolVersion;
 
     private final StreamSession session;
@@ -82,7 +82,7 @@ public class StreamingInboundHandler extends ChannelInboundHandlerAdapter
 
     private volatile boolean closed;
 
-    public StreamingInboundHandler(InetSocketAddress remoteAddress, int protocolVersion, @Nullable StreamSession session)
+    public StreamingInboundHandler(InetAddressAndPort remoteAddress, int protocolVersion, @Nullable StreamSession session)
     {
         this.remoteAddress = remoteAddress;
         this.protocolVersion = protocolVersion;
@@ -254,11 +254,11 @@ public class StreamingInboundHandler extends ChannelInboundHandlerAdapter
      */
     static class SessionIdentifier
     {
-        final InetAddress from;
+        final InetAddressAndPort from;
         final UUID planId;
         final int sessionIndex;
 
-        SessionIdentifier(InetAddress from, UUID planId, int sessionIndex)
+        SessionIdentifier(InetAddressAndPort from, UUID planId, int sessionIndex)
         {
             this.from = from;
             this.planId = planId;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java b/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java
index b9e6951..964fe10 100644
--- a/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java
+++ b/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.streaming.management;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.HashMap;
 import java.util.Map;
@@ -26,12 +25,14 @@ import javax.management.openmbean.*;
 
 import com.google.common.base.Throwables;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.ProgressInfo;
 
 public class ProgressInfoCompositeData
 {
     private static final String[] ITEM_NAMES = new String[]{"planId",
                                                             "peer",
+                                                            "peer storage port",
                                                             "sessionIndex",
                                                             "fileName",
                                                             "direction",
@@ -39,6 +40,7 @@ public class ProgressInfoCompositeData
                                                             "totalBytes"};
     private static final String[] ITEM_DESCS = new String[]{"String representation of Plan ID",
                                                             "Session peer",
+                                                            "Session peer storage port",
                                                             "Index of session",
                                                             "Name of the file",
                                                             "Direction('IN' or 'OUT')",
@@ -47,6 +49,7 @@ public class ProgressInfoCompositeData
     private static final OpenType<?>[] ITEM_TYPES = new OpenType[]{SimpleType.STRING,
                                                                    SimpleType.STRING,
                                                                    SimpleType.INTEGER,
+                                                                   SimpleType.INTEGER,
                                                                    SimpleType.STRING,
                                                                    SimpleType.STRING,
                                                                    SimpleType.LONG,
@@ -73,12 +76,13 @@ public class ProgressInfoCompositeData
     {
         Map<String, Object> valueMap = new HashMap<>();
         valueMap.put(ITEM_NAMES[0], planId.toString());
-        valueMap.put(ITEM_NAMES[1], progressInfo.peer.getHostAddress());
-        valueMap.put(ITEM_NAMES[2], progressInfo.sessionIndex);
-        valueMap.put(ITEM_NAMES[3], progressInfo.fileName);
-        valueMap.put(ITEM_NAMES[4], progressInfo.direction.name());
-        valueMap.put(ITEM_NAMES[5], progressInfo.currentBytes);
-        valueMap.put(ITEM_NAMES[6], progressInfo.totalBytes);
+        valueMap.put(ITEM_NAMES[1], progressInfo.peer.address.getHostAddress());
+        valueMap.put(ITEM_NAMES[2], progressInfo.peer.port);
+        valueMap.put(ITEM_NAMES[3], progressInfo.sessionIndex);
+        valueMap.put(ITEM_NAMES[4], progressInfo.fileName);
+        valueMap.put(ITEM_NAMES[5], progressInfo.direction.name());
+        valueMap.put(ITEM_NAMES[6], progressInfo.currentBytes);
+        valueMap.put(ITEM_NAMES[7], progressInfo.totalBytes);
         try
         {
             return new CompositeDataSupport(COMPOSITE_TYPE, valueMap);
@@ -94,12 +98,12 @@ public class ProgressInfoCompositeData
         Object[] values = cd.getAll(ITEM_NAMES);
         try
         {
-            return new ProgressInfo(InetAddress.getByName((String) values[1]),
-                                    (int) values[2],
-                                    (String) values[3],
-                                    ProgressInfo.Direction.valueOf((String)values[4]),
-                                    (long) values[5],
-                                    (long) values[6]);
+            return new ProgressInfo(InetAddressAndPort.getByNameOverrideDefaults((String) values[1], (Integer)values[2]),
+                                    (int) values[3],
+                                    (String) values[4],
+                                    ProgressInfo.Direction.valueOf((String)values[5]),
+                                    (long) values[6],
+                                    (long) values[7]);
         }
         catch (UnknownHostException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java b/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java
index 516582a..1c0d8c5 100644
--- a/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java
+++ b/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java
@@ -29,12 +29,15 @@ public class SessionCompleteEventCompositeData
 {
     private static final String[] ITEM_NAMES = new String[]{"planId",
                                                             "peer",
+                                                            "peer storage port",
                                                             "success"};
     private static final String[] ITEM_DESCS = new String[]{"Plan ID",
                                                             "Session peer",
+                                                            "Session peer storage port",
                                                             "Indicates whether session was successful"};
     private static final OpenType<?>[] ITEM_TYPES = new OpenType[]{SimpleType.STRING,
                                                                    SimpleType.STRING,
+                                                                   SimpleType.INTEGER,
                                                                    SimpleType.BOOLEAN};
 
     public static final CompositeType COMPOSITE_TYPE;
@@ -58,8 +61,9 @@ public class SessionCompleteEventCompositeData
     {
         Map<String, Object> valueMap = new HashMap<>();
         valueMap.put(ITEM_NAMES[0], event.planId.toString());
-        valueMap.put(ITEM_NAMES[1], event.peer.getHostAddress());
-        valueMap.put(ITEM_NAMES[2], event.success);
+        valueMap.put(ITEM_NAMES[1], event.peer.address.getHostAddress());
+        valueMap.put(ITEM_NAMES[2], event.peer.port);
+        valueMap.put(ITEM_NAMES[3], event.success);
         try
         {
             return new CompositeDataSupport(COMPOSITE_TYPE, valueMap);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java b/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java
index a6762a8..d20eaf5 100644
--- a/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java
+++ b/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.streaming.management;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 import javax.management.openmbean.*;
@@ -27,6 +26,7 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.SessionInfo;
 import org.apache.cassandra.streaming.StreamSession;
@@ -36,7 +36,9 @@ public class SessionInfoCompositeData
 {
     private static final String[] ITEM_NAMES = new String[]{"planId",
                                                             "peer",
+                                                            "peer_port",
                                                             "connecting",
+                                                            "connecting_port",
                                                             "receivingSummaries",
                                                             "sendingSummaries",
                                                             "state",
@@ -45,7 +47,9 @@ public class SessionInfoCompositeData
                                                             "sessionIndex"};
     private static final String[] ITEM_DESCS = new String[]{"Plan ID",
                                                             "Session peer",
+                                                            "Session peer storage port",
                                                             "Connecting address",
+                                                            "Connecting storage port",
                                                             "Summaries of receiving data",
                                                             "Summaries of sending data",
                                                             "Current session state",
@@ -61,7 +65,9 @@ public class SessionInfoCompositeData
         {
             ITEM_TYPES = new OpenType[]{SimpleType.STRING,
                                         SimpleType.STRING,
+                                        SimpleType.INTEGER,
                                         SimpleType.STRING,
+                                        SimpleType.INTEGER,
                                         ArrayType.getArrayType(StreamSummaryCompositeData.COMPOSITE_TYPE),
                                         ArrayType.getArrayType(StreamSummaryCompositeData.COMPOSITE_TYPE),
                                         SimpleType.STRING,
@@ -84,8 +90,10 @@ public class SessionInfoCompositeData
     {
         Map<String, Object> valueMap = new HashMap<>();
         valueMap.put(ITEM_NAMES[0], planId.toString());
-        valueMap.put(ITEM_NAMES[1], sessionInfo.peer.getHostAddress());
-        valueMap.put(ITEM_NAMES[2], sessionInfo.connecting.getHostAddress());
+        valueMap.put(ITEM_NAMES[1], sessionInfo.peer.address.getHostAddress());
+        valueMap.put(ITEM_NAMES[2], sessionInfo.peer.port);
+        valueMap.put(ITEM_NAMES[3], sessionInfo.connecting.address.getHostAddress());
+        valueMap.put(ITEM_NAMES[4], sessionInfo.connecting.port);
         Function<StreamSummary, CompositeData> fromStreamSummary = new Function<StreamSummary, CompositeData>()
         {
             public CompositeData apply(StreamSummary input)
@@ -93,9 +101,9 @@ public class SessionInfoCompositeData
                 return StreamSummaryCompositeData.toCompositeData(input);
             }
         };
-        valueMap.put(ITEM_NAMES[3], toArrayOfCompositeData(sessionInfo.receivingSummaries, fromStreamSummary));
-        valueMap.put(ITEM_NAMES[4], toArrayOfCompositeData(sessionInfo.sendingSummaries, fromStreamSummary));
-        valueMap.put(ITEM_NAMES[5], sessionInfo.state.name());
+        valueMap.put(ITEM_NAMES[5], toArrayOfCompositeData(sessionInfo.receivingSummaries, fromStreamSummary));
+        valueMap.put(ITEM_NAMES[6], toArrayOfCompositeData(sessionInfo.sendingSummaries, fromStreamSummary));
+        valueMap.put(ITEM_NAMES[7], sessionInfo.state.name());
         Function<ProgressInfo, CompositeData> fromProgressInfo = new Function<ProgressInfo, CompositeData>()
         {
             public CompositeData apply(ProgressInfo input)
@@ -103,9 +111,9 @@ public class SessionInfoCompositeData
                 return ProgressInfoCompositeData.toCompositeData(planId, input);
             }
         };
-        valueMap.put(ITEM_NAMES[6], toArrayOfCompositeData(sessionInfo.getReceivingFiles(), fromProgressInfo));
-        valueMap.put(ITEM_NAMES[7], toArrayOfCompositeData(sessionInfo.getSendingFiles(), fromProgressInfo));
-        valueMap.put(ITEM_NAMES[8], sessionInfo.sessionIndex);
+        valueMap.put(ITEM_NAMES[8], toArrayOfCompositeData(sessionInfo.getReceivingFiles(), fromProgressInfo));
+        valueMap.put(ITEM_NAMES[9], toArrayOfCompositeData(sessionInfo.getSendingFiles(), fromProgressInfo));
+        valueMap.put(ITEM_NAMES[10], sessionInfo.sessionIndex);
         try
         {
             return new CompositeDataSupport(COMPOSITE_TYPE, valueMap);
@@ -121,11 +129,11 @@ public class SessionInfoCompositeData
         assert cd.getCompositeType().equals(COMPOSITE_TYPE);
 
         Object[] values = cd.getAll(ITEM_NAMES);
-        InetAddress peer, connecting;
+        InetAddressAndPort peer, connecting;
         try
         {
-            peer = InetAddress.getByName((String) values[1]);
-            connecting = InetAddress.getByName((String) values[2]);
+            peer = InetAddressAndPort.getByNameOverrideDefaults((String) values[1], (Integer)values[2]);
+            connecting = InetAddressAndPort.getByNameOverrideDefaults((String) values[3], (Integer)values[4]);
         }
         catch (UnknownHostException e)
         {
@@ -139,11 +147,11 @@ public class SessionInfoCompositeData
             }
         };
         SessionInfo info = new SessionInfo(peer,
-                                           (int)values[8],
+                                           (int)values[10],
                                            connecting,
-                                           fromArrayOfCompositeData((CompositeData[]) values[3], toStreamSummary),
-                                           fromArrayOfCompositeData((CompositeData[]) values[4], toStreamSummary),
-                                           StreamSession.State.valueOf((String) values[5]));
+                                           fromArrayOfCompositeData((CompositeData[]) values[5], toStreamSummary),
+                                           fromArrayOfCompositeData((CompositeData[]) values[6], toStreamSummary),
+                                           StreamSession.State.valueOf((String) values[7]));
         Function<CompositeData, ProgressInfo> toProgressInfo = new Function<CompositeData, ProgressInfo>()
         {
             public ProgressInfo apply(CompositeData input)
@@ -151,11 +159,11 @@ public class SessionInfoCompositeData
                 return ProgressInfoCompositeData.fromCompositeData(input);
             }
         };
-        for (ProgressInfo progress : fromArrayOfCompositeData((CompositeData[]) values[6], toProgressInfo))
+        for (ProgressInfo progress : fromArrayOfCompositeData((CompositeData[]) values[8], toProgressInfo))
         {
             info.updateProgress(progress);
         }
-        for (ProgressInfo progress : fromArrayOfCompositeData((CompositeData[]) values[7], toProgressInfo))
+        for (ProgressInfo progress : fromArrayOfCompositeData((CompositeData[]) values[9], toProgressInfo))
         {
             info.updateProgress(progress);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 fedb971..13a3358 100644
--- a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -18,7 +18,6 @@
 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;
@@ -30,6 +29,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.TableId;
@@ -67,13 +67,13 @@ public class FileMessageHeader
     public final UUID pendingRepair;
     public final int sstableLevel;
     public final SerializationHeader.Component header;
-    public final InetAddress sender;
+    public final InetAddressAndPort sender;
 
     /* cached size value */
     private transient final long size;
 
     private FileMessageHeader(TableId tableId,
-                             InetAddress sender,
+                             InetAddressAndPort sender,
                              UUID planId,
                              int sessionIndex,
                              int sequenceNumber,
@@ -106,7 +106,7 @@ public class FileMessageHeader
     }
 
     public FileMessageHeader(TableId tableId,
-                             InetAddress sender,
+                             InetAddressAndPort sender,
                              UUID planId,
                              int sessionIndex,
                              int sequenceNumber,
@@ -218,7 +218,7 @@ public class FileMessageHeader
         public CompressionInfo serialize(FileMessageHeader header, DataOutputPlus out, int version) throws IOException
         {
             header.tableId.serialize(out);
-            CompactEndpointSerializationHelper.serialize(header.sender, out);
+            CompactEndpointSerializationHelper.streamingInstance.serialize(header.sender, out, version);
             UUIDSerializer.serializer.serialize(header.planId, out, version);
             out.writeInt(header.sessionIndex);
             out.writeInt(header.sequenceNumber);
@@ -252,7 +252,7 @@ public class FileMessageHeader
         public FileMessageHeader deserialize(DataInputPlus in, int version) throws IOException
         {
             TableId tableId = TableId.deserialize(in);
-            InetAddress sender = CompactEndpointSerializationHelper.deserialize(in);
+            InetAddressAndPort sender = CompactEndpointSerializationHelper.streamingInstance.deserialize(in, version);
             UUID planId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
             int sessionIndex = in.readInt();
             int sequenceNumber = in.readInt();
@@ -276,7 +276,7 @@ public class FileMessageHeader
         public long serializedSize(FileMessageHeader header, int version)
         {
             long size = header.tableId.serializedSize();
-            size += CompactEndpointSerializationHelper.serializedSize(header.sender);
+            size += CompactEndpointSerializationHelper.streamingInstance.serializedSize(header.sender, version);
             size += UUIDSerializer.serializer.serializedSize(header.planId, version);
             size += TypeSizes.sizeof(header.sessionIndex);
             size += TypeSizes.sizeof(header.sequenceNumber);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 f44b41c..8bbcc05 100644
--- a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
@@ -79,7 +79,7 @@ public class OutgoingFileMessage extends StreamMessage
         SSTableReader sstable = ref.get();
         filename = sstable.getFilename();
         this.header = new FileMessageHeader(sstable.metadata().id,
-                                            FBUtilities.getBroadcastAddress(),
+                                            FBUtilities.getBroadcastAddressAndPort(),
                                             session.planId(),
                                             session.sessionIndex(),
                                             sequenceNumber,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 68c6034..fced133 100644
--- a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
@@ -18,12 +18,12 @@
 package org.apache.cassandra.streaming.messages;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.UUID;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.StreamOperation;
@@ -39,7 +39,7 @@ public class StreamInitMessage extends StreamMessage
 {
     public static Serializer<StreamInitMessage> serializer = new StreamInitMessageSerializer();
 
-    public final InetAddress from;
+    public final InetAddressAndPort from;
     public final int sessionIndex;
     public final UUID planId;
     public final StreamOperation streamOperation;
@@ -48,7 +48,7 @@ public class StreamInitMessage extends StreamMessage
     public final UUID pendingRepair;
     public final PreviewKind previewKind;
 
-    public StreamInitMessage(InetAddress from, int sessionIndex, UUID planId, StreamOperation streamOperation, boolean keepSSTableLevel, UUID pendingRepair, PreviewKind previewKind)
+    public StreamInitMessage(InetAddressAndPort from, int sessionIndex, UUID planId, StreamOperation streamOperation, boolean keepSSTableLevel, UUID pendingRepair, PreviewKind previewKind)
     {
         super(Type.STREAM_INIT);
         this.from = from;
@@ -73,7 +73,7 @@ public class StreamInitMessage extends StreamMessage
     {
         public void serialize(StreamInitMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
         {
-            CompactEndpointSerializationHelper.serialize(message.from, out);
+            CompactEndpointSerializationHelper.streamingInstance.serialize(message.from, out, version);
             out.writeInt(message.sessionIndex);
             UUIDSerializer.serializer.serialize(message.planId, out, MessagingService.current_version);
             out.writeUTF(message.streamOperation.getDescription());
@@ -89,7 +89,7 @@ public class StreamInitMessage extends StreamMessage
 
         public StreamInitMessage deserialize(DataInputPlus in, int version, StreamSession session) throws IOException
         {
-            InetAddress from = CompactEndpointSerializationHelper.deserialize(in);
+            InetAddressAndPort from = CompactEndpointSerializationHelper.streamingInstance.deserialize(in, version);
             int sessionIndex = in.readInt();
             UUID planId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
             String description = in.readUTF();
@@ -102,7 +102,7 @@ public class StreamInitMessage extends StreamMessage
 
         public long serializedSize(StreamInitMessage message, int version)
         {
-            long size = CompactEndpointSerializationHelper.serializedSize(message.from);
+            long size = CompactEndpointSerializationHelper.streamingInstance.serializedSize(message.from, version);
             size += TypeSizes.sizeof(message.sessionIndex);
             size += UUIDSerializer.serializer.serializedSize(message.planId, MessagingService.current_version);
             size += TypeSizes.sizeof(message.streamOperation.getDescription());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 b56d292..cce686f 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
@@ -22,21 +22,21 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import io.netty.channel.Channel;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
 import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.StreamConnectionFactory;
 
 public class BulkLoadConnectionFactory extends DefaultConnectionFactory implements StreamConnectionFactory
 {
     private final boolean outboundBindAny;
-    private final int storagePort;
     private final int secureStoragePort;
     private final EncryptionOptions.ServerEncryptionOptions encryptionOptions;
 
-    public BulkLoadConnectionFactory(int storagePort, int secureStoragePort, EncryptionOptions.ServerEncryptionOptions encryptionOptions, boolean outboundBindAny)
+    public BulkLoadConnectionFactory(int secureStoragePort, EncryptionOptions.ServerEncryptionOptions encryptionOptions, boolean outboundBindAny)
     {
-        this.storagePort = storagePort;
         this.secureStoragePort = secureStoragePort;
         this.encryptionOptions = encryptionOptions != null && encryptionOptions.internode_encryption == EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none
                                  ? null
@@ -50,9 +50,9 @@ public class BulkLoadConnectionFactory extends DefaultConnectionFactory implemen
         // 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.
         int port = encryptionOptions != null && encryptionOptions.internode_encryption != EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none ?
-                   secureStoragePort : storagePort;
+                   secureStoragePort : connectionId.remote().port;
 
-        connectionId = connectionId.withNewConnectionAddress(new InetSocketAddress(connectionId.remote(), port));
+        connectionId = connectionId.withNewConnectionAddress(InetAddressAndPort.getByAddressOverrideDefaults(connectionId.remote().address, port));
         return createConnection(connectionId, protocolVersion, encryptionOptions);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/tools/BulkLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index 0812e53..545d1f7 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.tools;
 
 import java.io.IOException;
-import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.Set;
 import javax.net.ssl.SSLContext;
 
@@ -33,6 +33,7 @@ import org.apache.commons.cli.Options;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.io.sstable.SSTableLoader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.streaming.*;
 import org.apache.cassandra.utils.FBUtilities;
@@ -57,11 +58,12 @@ public class BulkLoader
                 new ExternalClient(
                         options.hosts,
                         options.nativePort,
-                        options.authProvider,
                         options.storagePort,
+                        options.authProvider,
                         options.sslStoragePort,
                         options.serverEncOptions,
-                        buildSSLOptions(options.clientEncOptions)),
+                        buildSSLOptions(options.clientEncOptions),
+                        options.allowServerPortDiscovery),
                         handler,
                         options.connectionsPerHost);
         DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(options.throttle);
@@ -124,7 +126,7 @@ public class BulkLoader
         private long peak = 0;
         private int totalFiles = 0;
 
-        private final Multimap<InetAddress, SessionInfo> sessionsByHost = HashMultimap.create();
+        private final Multimap<InetAddressAndPort, SessionInfo> sessionsByHost = HashMultimap.create();
 
         public ProgressIndicator()
         {
@@ -165,7 +167,7 @@ public class BulkLoader
 
                 boolean updateTotalFiles = totalFiles == 0;
                 // recalculate progress across all sessions in all hosts and display
-                for (InetAddress peer : sessionsByHost.keySet())
+                for (InetAddressAndPort peer : sessionsByHost.keySet())
                 {
                     sb.append("[").append(peer).append("]");
 
@@ -268,20 +270,19 @@ public class BulkLoader
 
     static class ExternalClient extends NativeSSTableLoaderClient
     {
-        private final int storagePort;
         private final int sslStoragePort;
         private final EncryptionOptions.ServerEncryptionOptions serverEncOptions;
 
-        public ExternalClient(Set<InetAddress> hosts,
-                              int port,
-                              AuthProvider authProvider,
+        public ExternalClient(Set<InetSocketAddress> hosts,
+                              int nativePort,
                               int storagePort,
+                              AuthProvider authProvider,
                               int sslStoragePort,
                               EncryptionOptions.ServerEncryptionOptions serverEncryptionOptions,
-                              SSLOptions sslOptions)
+                              SSLOptions sslOptions,
+                              boolean allowServerPortDiscovery)
         {
-            super(hosts, port, authProvider, sslOptions);
-            this.storagePort = storagePort;
+            super(hosts, nativePort, storagePort, authProvider, sslOptions, allowServerPortDiscovery);
             this.sslStoragePort = sslStoragePort;
             serverEncOptions = serverEncryptionOptions;
         }
@@ -289,7 +290,7 @@ public class BulkLoader
         @Override
         public StreamConnectionFactory getConnectionFactory()
         {
-            return new BulkLoadConnectionFactory(storagePort, sslStoragePort, serverEncOptions, false);
+            return new BulkLoadConnectionFactory(sslStoragePort, serverEncOptions, false);
         }
     }
 


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


[05/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java b/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
index 1b3fe03..78e87c1 100644
--- a/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
+++ b/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.dht;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -34,6 +33,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.AbstractNetworkTopologySnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -50,13 +50,13 @@ public class RangeFetchMapCalculatorTest
             //Odd IPs are in DC1 and Even are in DC2. Endpoints upto .14 will have unique racks and
             // then will be same for a set of three.
             @Override
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 return "RAC1";
             }
 
             @Override
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
                 if (getIPLastPart(endpoint) <= 50)
                     return DatabaseDescriptor.getLocalDataCenter();
@@ -66,9 +66,9 @@ public class RangeFetchMapCalculatorTest
                     return DatabaseDescriptor.getLocalDataCenter() + "Remote";
             }
 
-            private int getIPLastPart(InetAddress endpoint)
+            private int getIPLastPart(InetAddressAndPort endpoint)
             {
-                String str = endpoint.toString();
+                String str = endpoint.address.toString();
                 int index = str.lastIndexOf(".");
                 return Integer.parseInt(str.substring(index + 1).trim());
             }
@@ -78,7 +78,7 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testWithSingleSource() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3");
@@ -86,7 +86,7 @@ public class RangeFetchMapCalculatorTest
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.5");
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
         Assert.assertEquals(4, map.asMap().keySet().size());
@@ -95,7 +95,7 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testWithNonOverlappingSource() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1", "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3", "127.0.0.4");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.5", "127.0.0.6");
@@ -103,7 +103,7 @@ public class RangeFetchMapCalculatorTest
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.9", "127.0.0.10");
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
         Assert.assertEquals(5, map.asMap().keySet().size());
@@ -112,13 +112,13 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testWithRFThreeReplacement() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1", "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2", "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3", "127.0.0.4");
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
         //We should validate that it streamed from 3 unique sources
@@ -128,7 +128,7 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testForMultipleRoundsComputation() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3");
@@ -136,21 +136,21 @@ public class RangeFetchMapCalculatorTest
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.3", "127.0.0.2");
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
         //We should validate that it streamed from 2 unique sources
         Assert.assertEquals(2, map.asMap().keySet().size());
 
         assertArrays(Arrays.asList(generateNonTrivialRange(1, 10), generateNonTrivialRange(11, 20), generateNonTrivialRange(21, 30), generateNonTrivialRange(31, 40)),
-                map.asMap().get(InetAddress.getByName("127.0.0.3")));
-        assertArrays(Arrays.asList(generateNonTrivialRange(41, 50)), map.asMap().get(InetAddress.getByName("127.0.0.2")));
+                map.asMap().get(InetAddressAndPort.getByName("127.0.0.3")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(41, 50)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.2")));
     }
 
     @Test
     public void testForMultipleRoundsComputationWithLocalHost() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.1");
@@ -158,19 +158,19 @@ public class RangeFetchMapCalculatorTest
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.1", "127.0.0.2");
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
         //We should validate that it streamed from only non local host and only one range
         Assert.assertEquals(1, map.asMap().keySet().size());
 
-        assertArrays(Arrays.asList(generateNonTrivialRange(41, 50)), map.asMap().get(InetAddress.getByName("127.0.0.2")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(41, 50)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.2")));
     }
 
     @Test
     public void testForEmptyGraph() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.1");
@@ -178,7 +178,7 @@ public class RangeFetchMapCalculatorTest
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.1");
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         //All ranges map to local host so we will not stream anything.
         assertTrue(map.isEmpty());
     }
@@ -186,7 +186,7 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testWithNoSourceWithLocal() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1", "127.0.0.5");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3");
@@ -194,11 +194,11 @@ public class RangeFetchMapCalculatorTest
         //Return false for all except 127.0.0.5
         final RangeStreamer.ISourceFilter filter = new RangeStreamer.ISourceFilter()
         {
-            public boolean shouldInclude(InetAddress endpoint)
+            public boolean shouldInclude(InetAddressAndPort endpoint)
             {
                 try
                 {
-                    if (endpoint.equals(InetAddress.getByName("127.0.0.5")))
+                    if (endpoint.equals(InetAddressAndPort.getByName("127.0.0.5")))
                         return false;
                     else
                         return true;
@@ -211,28 +211,28 @@ public class RangeFetchMapCalculatorTest
         };
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Arrays.asList(filter), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
 
         validateRange(rangesWithSources, map);
 
         //We should validate that it streamed from only non local host and only one range
         Assert.assertEquals(2, map.asMap().keySet().size());
 
-        assertArrays(Arrays.asList(generateNonTrivialRange(11, 20)), map.asMap().get(InetAddress.getByName("127.0.0.2")));
-        assertArrays(Arrays.asList(generateNonTrivialRange(21, 30)), map.asMap().get(InetAddress.getByName("127.0.0.3")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(11, 20)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.2")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(21, 30)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.3")));
     }
 
     @Test (expected = IllegalStateException.class)
     public void testWithNoLiveSource() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.5");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3");
 
         final RangeStreamer.ISourceFilter allDeadFilter = new RangeStreamer.ISourceFilter()
         {
-            public boolean shouldInclude(InetAddress endpoint)
+            public boolean shouldInclude(InetAddressAndPort endpoint)
             {
                 return false;
             }
@@ -245,25 +245,25 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testForLocalDC() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1", "127.0.0.3", "127.0.0.53");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.1", "127.0.0.3", "127.0.0.57");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.2", "127.0.0.59", "127.0.0.61");
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<>(), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
         Assert.assertEquals(2, map.asMap().size());
 
         //Should have streamed from local DC endpoints
-        assertArrays(Arrays.asList(generateNonTrivialRange(21, 30)), map.asMap().get(InetAddress.getByName("127.0.0.2")));
-        assertArrays(Arrays.asList(generateNonTrivialRange(1, 10), generateNonTrivialRange(11, 20)), map.asMap().get(InetAddress.getByName("127.0.0.3")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(21, 30)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.2")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(1, 10), generateNonTrivialRange(11, 20)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.3")));
     }
 
     @Test
     public void testForRemoteDC() throws Exception
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3", "127.0.0.55");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.2", "127.0.0.59");
@@ -271,11 +271,11 @@ public class RangeFetchMapCalculatorTest
         //Reject only 127.0.0.3 and accept everyone else
         final RangeStreamer.ISourceFilter localHostFilter = new RangeStreamer.ISourceFilter()
         {
-            public boolean shouldInclude(InetAddress endpoint)
+            public boolean shouldInclude(InetAddressAndPort endpoint)
             {
                 try
                 {
-                    if (endpoint.equals(InetAddress.getByName("127.0.0.3")))
+                    if (endpoint.equals(InetAddressAndPort.getByName("127.0.0.3")))
                         return false;
                     else
                         return true;
@@ -288,20 +288,20 @@ public class RangeFetchMapCalculatorTest
         };
 
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Arrays.asList(localHostFilter), "Test");
-        Multimap<InetAddress, Range<Token>> map = calculator.getRangeFetchMap();
+        Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
         Assert.assertEquals(3, map.asMap().size());
 
         //Should have streamed from remote DC endpoint
-        assertArrays(Arrays.asList(generateNonTrivialRange(1, 10)), map.asMap().get(InetAddress.getByName("127.0.0.51")));
-        assertArrays(Arrays.asList(generateNonTrivialRange(11, 20)), map.asMap().get(InetAddress.getByName("127.0.0.55")));
-        assertArrays(Arrays.asList(generateNonTrivialRange(21, 30)), map.asMap().get(InetAddress.getByName("127.0.0.2")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(1, 10)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.51")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(11, 20)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.55")));
+        assertArrays(Arrays.asList(generateNonTrivialRange(21, 30)), map.asMap().get(InetAddressAndPort.getByName("127.0.0.2")));
     }
 
     @Test
     public void testTrivialRanges() throws UnknownHostException
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         // add non-trivial ranges
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3", "127.0.0.55");
@@ -309,17 +309,17 @@ public class RangeFetchMapCalculatorTest
         // and a trivial one:
         addTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51");
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Collections.emptyList(), "Test");
-        Multimap<InetAddress, Range<Token>> optMap = calculator.getRangeFetchMapForNonTrivialRanges();
-        Multimap<InetAddress, Range<Token>> trivialMap = calculator.getRangeFetchMapForTrivialRanges(optMap);
-        assertTrue(trivialMap.get(InetAddress.getByName("127.0.0.3")).contains(generateTrivialRange(1,10)) ^
-                   trivialMap.get(InetAddress.getByName("127.0.0.51")).contains(generateTrivialRange(1,10)));
+        Multimap<InetAddressAndPort, Range<Token>> optMap = calculator.getRangeFetchMapForNonTrivialRanges();
+        Multimap<InetAddressAndPort, Range<Token>> trivialMap = calculator.getRangeFetchMapForTrivialRanges(optMap);
+        assertTrue(trivialMap.get(InetAddressAndPort.getByName("127.0.0.3")).contains(generateTrivialRange(1,10)) ^
+                   trivialMap.get(InetAddressAndPort.getByName("127.0.0.51")).contains(generateTrivialRange(1,10)));
         assertFalse(optMap.containsKey(generateTrivialRange(1, 10)));
     }
 
     @Test(expected = IllegalStateException.class)
     public void testNotEnoughEndpointsForTrivialRange() throws UnknownHostException
     {
-        Multimap<Range<Token>, InetAddress> rangesWithSources = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
         // add non-trivial ranges
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3", "127.0.0.55");
@@ -329,11 +329,11 @@ public class RangeFetchMapCalculatorTest
 
         RangeStreamer.ISourceFilter filter = new RangeStreamer.ISourceFilter()
         {
-            public boolean shouldInclude(InetAddress endpoint)
+            public boolean shouldInclude(InetAddressAndPort endpoint)
             {
                 try
                 {
-                    if (endpoint.equals(InetAddress.getByName("127.0.0.3")))
+                    if (endpoint.equals(InetAddressAndPort.getByName("127.0.0.3")))
                         return false;
                 }
                 catch (UnknownHostException e)
@@ -344,8 +344,8 @@ public class RangeFetchMapCalculatorTest
             }
         };
         RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Collections.singleton(filter), "Test");
-        Multimap<InetAddress, Range<Token>> optMap = calculator.getRangeFetchMapForNonTrivialRanges();
-        Multimap<InetAddress, Range<Token>> trivialMap = calculator.getRangeFetchMapForTrivialRanges(optMap);
+        Multimap<InetAddressAndPort, Range<Token>> optMap = calculator.getRangeFetchMapForNonTrivialRanges();
+        Multimap<InetAddressAndPort, Range<Token>> trivialMap = calculator.getRangeFetchMapForTrivialRanges(optMap);
 
     }
 
@@ -355,35 +355,35 @@ public class RangeFetchMapCalculatorTest
         assertTrue(result.containsAll(expected));
     }
 
-    private void validateRange(Multimap<Range<Token>, InetAddress> rangesWithSources, Multimap<InetAddress, Range<Token>> result)
+    private void validateRange(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources, Multimap<InetAddressAndPort, Range<Token>> result)
     {
-        for (Map.Entry<InetAddress, Range<Token>> entry : result.entries())
+        for (Map.Entry<InetAddressAndPort, Range<Token>> entry : result.entries())
         {
             assertTrue(rangesWithSources.get(entry.getValue()).contains(entry.getKey()));
         }
     }
 
-    private void addNonTrivialRangeAndSources(Multimap<Range<Token>, InetAddress> rangesWithSources, int left, int right, String... hosts) throws UnknownHostException
+    private void addNonTrivialRangeAndSources(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources, int left, int right, String... hosts) throws UnknownHostException
     {
-        for (InetAddress endpoint : makeAddrs(hosts))
+        for (InetAddressAndPort endpoint : makeAddrs(hosts))
         {
             rangesWithSources.put(generateNonTrivialRange(left, right), endpoint);
         }
     }
 
-    private void addTrivialRangeAndSources(Multimap<Range<Token>, InetAddress> rangesWithSources, int left, int right, String... hosts) throws UnknownHostException
+    private void addTrivialRangeAndSources(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources, int left, int right, String... hosts) throws UnknownHostException
     {
-        for (InetAddress endpoint : makeAddrs(hosts))
+        for (InetAddressAndPort endpoint : makeAddrs(hosts))
         {
             rangesWithSources.put(generateTrivialRange(left, right), endpoint);
         }
     }
 
-    private Collection<InetAddress> makeAddrs(String... hosts) throws UnknownHostException
+    private Collection<InetAddressAndPort> makeAddrs(String... hosts) throws UnknownHostException
     {
-        ArrayList<InetAddress> addrs = new ArrayList<InetAddress>(hosts.length);
+        ArrayList<InetAddressAndPort> addrs = new ArrayList<>(hosts.length);
         for (String host : hosts)
-            addrs.add(InetAddress.getByName(host));
+            addrs.add(InetAddressAndPort.getByName(host));
         return addrs;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
index b5f8036..ad5f8f5 100644
--- a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.dht;
 
-import java.net.InetAddress;
 import java.util.Collections;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.DefaultConnectionFactory;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.StreamEvent;
@@ -50,7 +50,7 @@ public class StreamStateStoreTest
         Token.TokenFactory factory = p.getTokenFactory();
         Range<Token> range = new Range<>(factory.fromString("0"), factory.fromString("100"));
 
-        InetAddress local = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
         StreamSession session = new StreamSession(local, local, new DefaultConnectionFactory(), 0, true, null, PreviewKind.NONE);
         session.addStreamRequest("keyspace1", Collections.singleton(range), Collections.singleton("cf"));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/gms/ArrivalWindowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/ArrivalWindowTest.java b/test/unit/org/apache/cassandra/gms/ArrivalWindowTest.java
index 511511b..ea59300 100644
--- a/test/unit/org/apache/cassandra/gms/ArrivalWindowTest.java
+++ b/test/unit/org/apache/cassandra/gms/ArrivalWindowTest.java
@@ -25,9 +25,8 @@ import static org.junit.Assert.*;
 
 import org.junit.Test;
 
-import java.lang.RuntimeException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.utils.FBUtilities;
 
 public class ArrivalWindowTest
 {
@@ -36,15 +35,7 @@ public class ArrivalWindowTest
     {
         final ArrivalWindow windowWithNano = new ArrivalWindow(4);
         final long toNano = 1000000L;
-        InetAddress ep;
-        try
-        {
-            ep = InetAddress.getLocalHost();
-        }
-        catch (UnknownHostException e)
-        {
-            throw new RuntimeException(e);
-        }
+        InetAddressAndPort ep = FBUtilities.getLocalAddressAndPort();
         windowWithNano.add(111 * toNano, ep);
         windowWithNano.add(222 * toNano, ep);
         windowWithNano.add(333 * toNano, ep);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/gms/EndpointStateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/EndpointStateTest.java b/test/unit/org/apache/cassandra/gms/EndpointStateTest.java
index 2453fe8..6e0cc75 100644
--- a/test/unit/org/apache/cassandra/gms/EndpointStateTest.java
+++ b/test/unit/org/apache/cassandra/gms/EndpointStateTest.java
@@ -72,7 +72,7 @@ public class EndpointStateTest
             public void run()
             {
                 state.addApplicationState(ApplicationState.TOKENS, valueFactory.tokens(tokens));
-                state.addApplicationState(ApplicationState.STATUS, valueFactory.normal(tokens));
+                state.addApplicationState(ApplicationState.STATUS_WITH_PORT, valueFactory.normal(tokens));
             }
         });
 
@@ -86,7 +86,7 @@ public class EndpointStateTest
                     for (Map.Entry<ApplicationState, VersionedValue> entry : state.states())
                         values.put(entry.getKey(), entry.getValue());
 
-                    if (values.containsKey(ApplicationState.STATUS) && !values.containsKey(ApplicationState.TOKENS))
+                    if (values.containsKey(ApplicationState.STATUS_WITH_PORT) && !values.containsKey(ApplicationState.TOKENS))
                     {
                         numFailures.incrementAndGet();
                         System.out.println(String.format("Failed: %s", values));
@@ -129,7 +129,7 @@ public class EndpointStateTest
             {
                 Map<ApplicationState, VersionedValue> states = new EnumMap<>(ApplicationState.class);
                 states.put(ApplicationState.TOKENS, valueFactory.tokens(tokens));
-                states.put(ApplicationState.STATUS, valueFactory.normal(tokens));
+                states.put(ApplicationState.STATUS_WITH_PORT, valueFactory.normal(tokens));
                 state.addApplicationStates(states);
             }
         });
@@ -158,7 +158,7 @@ public class EndpointStateTest
         for (Map.Entry<ApplicationState, VersionedValue> entry : states)
             values.put(entry.getKey(), entry.getValue());
 
-        assertTrue(values.containsKey(ApplicationState.STATUS));
+        assertTrue(values.containsKey(ApplicationState.STATUS_WITH_PORT));
         assertTrue(values.containsKey(ApplicationState.TOKENS));
         assertTrue(values.containsKey(ApplicationState.INTERNAL_IP));
         assertTrue(values.containsKey(ApplicationState.HOST_ID));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
index dca00b1..554e894 100644
--- a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
+++ b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -33,6 +32,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
 
@@ -60,7 +60,7 @@ public class FailureDetectorTest
 
         ArrayList<Token> endpointTokens = new ArrayList<>();
         ArrayList<Token> keyTokens = new ArrayList<>();
-        List<InetAddress> hosts = new ArrayList<>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
         List<UUID> hostIds = new ArrayList<>();
 
         // we want to convict if there is any heartbeat data present in the FD
@@ -69,12 +69,12 @@ public class FailureDetectorTest
         // create a ring of 2 nodes
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 3);
 
-        InetAddress leftHost = hosts.get(1);
+        InetAddressAndPort leftHost = hosts.get(1);
 
         FailureDetector.instance.report(leftHost);
 
         // trigger handleStateLeft in StorageService
-        ss.onChange(leftHost, ApplicationState.STATUS,
+        ss.onChange(leftHost, ApplicationState.STATUS_WITH_PORT,
                     valueFactory.left(Collections.singleton(endpointTokens.get(1)), Gossiper.computeExpireTime()));
 
         // confirm that handleStateLeft was called and leftEndpoint was removed from TokenMetadata

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/GossipDigestTest.java b/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
index 3191b03..cb67a54 100644
--- a/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
@@ -26,8 +26,7 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 
-import java.net.InetAddress;
-
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.junit.Test;
 
@@ -36,7 +35,7 @@ public class GossipDigestTest
     @Test
     public void test() throws IOException
     {
-        InetAddress endpoint = InetAddress.getByName("127.0.0.1");
+        InetAddressAndPort endpoint = InetAddressAndPort.getByName("127.0.0.1");
         int generation = 0;
         int maxVersion = 123;
         GossipDigest expected = new GossipDigest(endpoint, generation, maxVersion);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/gms/GossiperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index 83f12d1..8c65cb4 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
@@ -33,6 +32,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
 
@@ -49,7 +49,7 @@ public class GossiperTest
     TokenMetadata tmd = StorageService.instance.getTokenMetadata();
     ArrayList<Token> endpointTokens = new ArrayList<>();
     ArrayList<Token> keyTokens = new ArrayList<>();
-    List<InetAddress> hosts = new ArrayList<>();
+    List<InetAddressAndPort> hosts = new ArrayList<>();
     List<UUID> hostIds = new ArrayList<>();
 
     @Before
@@ -62,7 +62,7 @@ public class GossiperTest
     public void testLargeGenerationJump() throws UnknownHostException, InterruptedException
     {
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 2);
-        InetAddress remoteHostAddress = hosts.get(1);
+        InetAddressAndPort remoteHostAddress = hosts.get(1);
 
         EndpointState initialRemoteState = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress);
         HeartBeatState initialRemoteHeartBeat = initialRemoteState.getHeartBeatState();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java b/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
index 0939db0..833ee8b 100644
--- a/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
+++ b/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -35,6 +34,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageService;
 import org.jboss.byteman.contrib.bmunit.BMRule;
 import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
@@ -66,7 +66,7 @@ public class PendingRangeCalculatorServiceTest
             action = "org.apache.cassandra.gms.PendingRangeCalculatorServiceTest.calculationLock.lock()")
     public void testDelayedResponse() throws UnknownHostException, InterruptedException
     {
-        InetAddress otherNodeAddr = InetAddress.getByName("127.0.0.2");
+        InetAddressAndPort otherNodeAddr = InetAddressAndPort.getByName("127.0.0.2");
         UUID otherHostId = UUID.randomUUID();
 
         // introduce node for first major state change
@@ -112,7 +112,7 @@ public class PendingRangeCalculatorServiceTest
         }
     }
 
-    private Map<InetAddress, EndpointState> getStates(InetAddress otherNodeAddr, UUID hostId, int ver, boolean bootstrapping)
+    private Map<InetAddressAndPort, EndpointState> getStates(InetAddressAndPort otherNodeAddr, UUID hostId, int ver, boolean bootstrapping)
     {
         HeartBeatState hb = new HeartBeatState(1, ver);
         EndpointState state = new EndpointState(hb);
@@ -125,7 +125,7 @@ public class PendingRangeCalculatorServiceTest
         state.addApplicationState(ApplicationState.HOST_ID, StorageService.instance.valueFactory.hostId(hostId));
         state.addApplicationState(ApplicationState.NET_VERSION, StorageService.instance.valueFactory.networkVersion());
 
-        Map<InetAddress, EndpointState> states = new HashMap<>();
+        Map<InetAddressAndPort, EndpointState> states = new HashMap<>();
         states.put(otherNodeAddr, state);
         return states;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/gms/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/SerializationsTest.java b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
index 0df266f..90ce10b 100644
--- a/test/unit/org/apache/cassandra/gms/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
@@ -24,6 +24,7 @@ import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -31,7 +32,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -78,9 +78,9 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private void testGossipDigestWrite() throws IOException
     {
-        Map<InetAddress, EndpointState> states = new HashMap<InetAddress, EndpointState>();
-        states.put(InetAddress.getByName("127.0.0.1"), Statics.EndpointSt);
-        states.put(InetAddress.getByName("127.0.0.2"), Statics.EndpointSt);
+        Map<InetAddressAndPort, EndpointState> states = new HashMap<>();
+        states.put(InetAddressAndPort.getByName("127.0.0.1"), Statics.EndpointSt);
+        states.put(InetAddressAndPort.getByName("127.0.0.2"), Statics.EndpointSt);
         GossipDigestAck ack = new GossipDigestAck(Statics.Digests, states);
         GossipDigestAck2 ack2 = new GossipDigestAck2(states);
         GossipDigestSyn syn = new GossipDigestSyn("Not a real cluster name",
@@ -132,9 +132,9 @@ public class SerializationsTest extends AbstractSerializationsTester
         {
             HeartbeatSt.updateHeartBeat();
             EndpointSt.addApplicationState(ApplicationState.LOAD, vv0);
-            EndpointSt.addApplicationState(ApplicationState.STATUS, vv1);
+            EndpointSt.addApplicationState(ApplicationState.STATUS_WITH_PORT, vv1);
             for (int i = 0; i < 100; i++)
-                Digests.add(new GossipDigest(FBUtilities.getBroadcastAddress(), 100 + i, 1000 + 2 * i));
+                Digests.add(new GossipDigest(FBUtilities.getBroadcastAddressAndPort(), 100 + i, 1000 + 2 * i));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/hints/HintTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintTest.java b/test/unit/org/apache/cassandra/hints/HintTest.java
index 23189e1..e8960f5 100644
--- a/test/unit/org/apache/cassandra/hints/HintTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintTest.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.hints;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.UUID;
 
@@ -39,6 +38,7 @@ import org.apache.cassandra.dht.BootStrapper;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.net.MessageIn;
@@ -81,7 +81,7 @@ public class HintTest
     public void resetGcGraceSeconds()
     {
         TokenMetadata tokenMeta = StorageService.instance.getTokenMetadata();
-        InetAddress local = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
         tokenMeta.clearUnsafe();
         tokenMeta.updateHostId(UUID.randomUUID(), local);
         tokenMeta.updateNormalTokens(BootStrapper.getRandomTokens(tokenMeta, 1), local);
@@ -230,8 +230,8 @@ public class HintTest
 
         // Prepare metadata with injected stale endpoint serving the mutation key.
         TokenMetadata tokenMeta = StorageService.instance.getTokenMetadata();
-        InetAddress local = FBUtilities.getBroadcastAddress();
-        InetAddress endpoint = InetAddress.getByName("1.1.1.1");
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
+        InetAddressAndPort endpoint = InetAddressAndPort.getByName("1.1.1.1");
         UUID localId = StorageService.instance.getLocalHostUUID();
         UUID targetId = UUID.randomUUID();
         tokenMeta.updateHostId(targetId, endpoint);
@@ -271,8 +271,8 @@ public class HintTest
 
         // Prepare metadata with injected stale endpoint.
         TokenMetadata tokenMeta = StorageService.instance.getTokenMetadata();
-        InetAddress local = FBUtilities.getBroadcastAddress();
-        InetAddress endpoint = InetAddress.getByName("1.1.1.1");
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
+        InetAddressAndPort endpoint = InetAddressAndPort.getByName("1.1.1.1");
         UUID localId = StorageService.instance.getLocalHostUUID();
         UUID targetId = UUID.randomUUID();
         tokenMeta.updateHostId(targetId, endpoint);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/hints/HintsServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintsServiceTest.java b/test/unit/org/apache/cassandra/hints/HintsServiceTest.java
index 2da8993..b71140f 100644
--- a/test/unit/org/apache/cassandra/hints/HintsServiceTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintsServiceTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.hints;
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.ExecutionException;
@@ -34,6 +33,7 @@ import org.junit.Test;
 
 import com.datastax.driver.core.utils.MoreFutures;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.DecoratedKey;
@@ -182,7 +182,7 @@ public class HintsServiceTest
     private MockMessagingSpy sendHintsAndResponses(int noOfHints, int noOfResponses)
     {
         // create spy for hint messages, but only create responses for noOfResponses hints
-        MessageIn<HintResponse> messageIn = MessageIn.create(FBUtilities.getBroadcastAddress(),
+        MessageIn<HintResponse> messageIn = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(),
                 HintResponse.instance,
                 Collections.emptyMap(),
                 MessagingService.Verb.REQUEST_RESPONSE,
@@ -217,17 +217,17 @@ public class HintsServiceTest
     {
         private boolean isAlive = true;
 
-        public boolean isAlive(InetAddress ep)
+        public boolean isAlive(InetAddressAndPort ep)
         {
             return isAlive;
         }
 
-        public void interpret(InetAddress ep)
+        public void interpret(InetAddressAndPort ep)
         {
             throw new UnsupportedOperationException();
         }
 
-        public void report(InetAddress ep)
+        public void report(InetAddressAndPort ep)
         {
             throw new UnsupportedOperationException();
         }
@@ -242,12 +242,12 @@ public class HintsServiceTest
             throw new UnsupportedOperationException();
         }
 
-        public void remove(InetAddress ep)
+        public void remove(InetAddressAndPort ep)
         {
             throw new UnsupportedOperationException();
         }
 
-        public void forceConviction(InetAddress ep)
+        public void forceConviction(InetAddressAndPort ep)
         {
             throw new UnsupportedOperationException();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index d63d6c3..dbb929d 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -650,7 +650,7 @@ public class CQLSSTableWriterTest
             {
                 this.keyspace = keyspace;
                 for (Range<Token> range : StorageService.instance.getLocalRanges(ks))
-                    addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
+                    addRangeForEndpoint(range, FBUtilities.getBroadcastAddressAndPort());
             }
 
             public TableMetadataRef getTableMetadata(String cfName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 6e7d173..8fc69bf 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -215,7 +215,7 @@ public class LegacySSTableTest
         details.add(new StreamSession.SSTableStreamingSections(sstable.ref(),
                                                                sstable.getPositionsForRanges(ranges),
                                                                sstable.estimatedKeysForRanges(ranges)));
-        new StreamPlan(StreamOperation.OTHER).transferFiles(FBUtilities.getBroadcastAddress(), details)
+        new StreamPlan(StreamOperation.OTHER).transferFiles(FBUtilities.getBroadcastAddressAndPort(), details)
                                   .execute().get();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index 4f3739f..430b7c2 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -104,7 +104,7 @@ public class SSTableLoaderTest
         {
             this.keyspace = keyspace;
             for (Range<Token> range : StorageService.instance.getLocalRanges(KEYSPACE1))
-                addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
+                addRangeForEndpoint(range, FBUtilities.getBroadcastAddressAndPort());
         }
 
         public TableMetadataRef getTableMetadata(String tableName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 34f8204..391f789 100644
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.locator;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.EnumMap;
 import java.util.Map;
 
@@ -77,8 +76,8 @@ public class CloudstackSnitchTest
     {
         az = "ch-gva-1";
         CloudstackSnitch snitch = new TestCloudstackSnitch();
-        InetAddress local = InetAddress.getByName("127.0.0.1");
-        InetAddress nonlocal = InetAddress.getByName("127.0.0.7");
+        InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1");
+        InetAddressAndPort nonlocal = InetAddressAndPort.getByName("127.0.0.7");
 
         Gossiper.instance.addSavedEndpoint(nonlocal);
         Map<ApplicationState, VersionedValue> stateMap = new EnumMap<>(ApplicationState.class);
@@ -99,7 +98,7 @@ public class CloudstackSnitchTest
     {
         az = "ch-gva-1";
         CloudstackSnitch snitch = new TestCloudstackSnitch();
-        InetAddress local = InetAddress.getByName("127.0.0.1");
+        InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1");
 
         assertEquals("ch-gva", snitch.getDatacenter(local));
         assertEquals("1", snitch.getRack(local));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
index 8a59a4a..bf1e4c2 100644
--- a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.locator;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.*;
 
 import org.junit.BeforeClass;
@@ -41,7 +40,7 @@ public class DynamicEndpointSnitchTest
         DatabaseDescriptor.daemonInitialization();
     }
 
-    private static void setScores(DynamicEndpointSnitch dsnitch,  int rounds, List<InetAddress> hosts, Integer... scores) throws InterruptedException
+    private static void setScores(DynamicEndpointSnitch dsnitch, int rounds, List<InetAddressAndPort> hosts, Integer... scores) throws InterruptedException
     {
         for (int round = 0; round < rounds; round++)
         {
@@ -58,16 +57,16 @@ public class DynamicEndpointSnitchTest
         StorageService.instance.unsafeInitialize();
         SimpleSnitch ss = new SimpleSnitch();
         DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
-        InetAddress self = FBUtilities.getBroadcastAddress();
-        InetAddress host1 = InetAddress.getByName("127.0.0.2");
-        InetAddress host2 = InetAddress.getByName("127.0.0.3");
-        InetAddress host3 = InetAddress.getByName("127.0.0.4");
-        InetAddress host4 = InetAddress.getByName("127.0.0.5");
-        List<InetAddress> hosts = Arrays.asList(host1, host2, host3);
+        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
+        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
+        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
+        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
+        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
+        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
 
         // first, make all hosts equal
         setScores(dsnitch, 1, hosts, 10, 10, 10);
-        List<InetAddress> order = Arrays.asList(host1, host2, host3);
+        List<InetAddressAndPort> order = Arrays.asList(host1, host2, host3);
         assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
 
         // make host1 a little worse

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 09973a8..182ff1a 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -20,7 +20,7 @@ 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;
 
@@ -72,8 +72,8 @@ public class EC2SnitchTest
     {
         az = "us-east-1d";
         Ec2Snitch snitch = new TestEC2Snitch();
-        InetAddress local = InetAddress.getByName("127.0.0.1");
-        InetAddress nonlocal = InetAddress.getByName("127.0.0.7");
+        InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1");
+        InetAddressAndPort nonlocal = InetAddressAndPort.getByName("127.0.0.7");
 
         Gossiper.instance.addSavedEndpoint(nonlocal);
         Map<ApplicationState, VersionedValue> stateMap = new EnumMap<>(ApplicationState.class);
@@ -93,7 +93,7 @@ public class EC2SnitchTest
     {
         az = "us-east-2d";
         Ec2Snitch snitch = new TestEC2Snitch();
-        InetAddress local = InetAddress.getByName("127.0.0.1");
+        InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1");
         assertEquals("us-east-2", snitch.getDatacenter(local));
         assertEquals("2d", snitch.getRack(local));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
index c63044c..ea6fa2d 100644
--- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.locator;
 
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.EnumMap;
 import java.util.Map;
 
@@ -72,8 +71,8 @@ public class GoogleCloudSnitchTest
     {
         az = "us-central1-a";
         GoogleCloudSnitch snitch = new TestGoogleCloudSnitch();
-        InetAddress local = InetAddress.getByName("127.0.0.1");
-        InetAddress nonlocal = InetAddress.getByName("127.0.0.7");
+        InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1");
+        InetAddressAndPort nonlocal = InetAddressAndPort.getByName("127.0.0.7");
 
         Gossiper.instance.addSavedEndpoint(nonlocal);
         Map<ApplicationState, VersionedValue> stateMap = new EnumMap<>(ApplicationState.class);
@@ -93,7 +92,7 @@ public class GoogleCloudSnitchTest
     {
         az = "asia-east1-a";
         GoogleCloudSnitch snitch = new TestGoogleCloudSnitch();
-        InetAddress local = InetAddress.getByName("127.0.0.1");
+        InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1");
         assertEquals("asia-east1", snitch.getDatacenter(local));
         assertEquals("a", snitch.getRack(local));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java b/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java
index 77734f7..da26003 100644
--- a/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java
@@ -18,9 +18,8 @@
 
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
+import java.net.UnknownHostException;
 
-import com.google.common.net.InetAddresses;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -45,7 +44,15 @@ public class GossipingPropertyFileSnitchTest
                                      final String endpointString, final String expectedDatacenter,
                                      final String expectedRack)
     {
-        final InetAddress endpoint = InetAddresses.forString(endpointString);
+        final InetAddressAndPort endpoint;
+        try
+        {
+            endpoint = InetAddressAndPort.getByName(endpointString);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
         assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint));
         assertEquals(expectedRack, snitch.getRack(endpoint));
     }
@@ -54,6 +61,6 @@ public class GossipingPropertyFileSnitchTest
     public void testLoadConfig() throws Exception
     {
         final GossipingPropertyFileSnitch snitch = new GossipingPropertyFileSnitch();
-        checkEndpoint(snitch, FBUtilities.getBroadcastAddress().getHostAddress(), "DC1", "RAC1");
+        checkEndpoint(snitch, FBUtilities.getBroadcastAddressAndPort().toString(), "DC1", "RAC1");
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/InetAddressAndPortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/InetAddressAndPortTest.java b/test/unit/org/apache/cassandra/locator/InetAddressAndPortTest.java
new file mode 100644
index 0000000..c32b9a9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/locator/InetAddressAndPortTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.locator;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class InetAddressAndPortTest
+{
+    private static interface ThrowingRunnable
+    {
+        public void run() throws Throwable;
+    }
+
+    @Test
+    public void getByNameIPv4Test() throws Exception
+    {
+        //Negative port
+        shouldThrow(() -> InetAddressAndPort.getByName("127.0.0.1:-1"), IllegalArgumentException.class);
+        //Too large port
+        shouldThrow(() -> InetAddressAndPort.getByName("127.0.0.1:65536"), IllegalArgumentException.class);
+
+        //bad address, caught by InetAddress
+        shouldThrow(() -> InetAddressAndPort.getByName("127.0.0.1.0"), UnknownHostException.class);
+
+        //Test default port
+        InetAddressAndPort address = InetAddressAndPort.getByName("127.0.0.1");
+        assertEquals(InetAddress.getByName("127.0.0.1"), address.address);
+        assertEquals(InetAddressAndPort.defaultPort, address.port);
+
+        //Test overriding default port
+        address = InetAddressAndPort.getByName("127.0.0.1:42");
+        assertEquals(InetAddress.getByName("127.0.0.1"), address.address);
+        assertEquals(42, address.port);
+    }
+
+    @Test
+    public void getByNameIPv6Test() throws Exception
+    {
+        //Negative port
+        shouldThrow(() -> InetAddressAndPort.getByName("[2001:0db8:0000:0000:0000:ff00:0042:8329]:-1"), IllegalArgumentException.class);
+        //Too large port
+        shouldThrow(() -> InetAddressAndPort.getByName("[2001:0db8:0000:0000:0000:ff00:0042:8329]:65536"), IllegalArgumentException.class);
+
+        //bad address, caught by InetAddress
+        shouldThrow(() -> InetAddressAndPort.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329:8329"), UnknownHostException.class);
+
+        //Test default port
+        InetAddressAndPort address = InetAddressAndPort.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329");
+        assertEquals(InetAddress.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329"), address.address);
+        assertEquals(InetAddressAndPort.defaultPort, address.port);
+
+        //Test overriding default port
+        address = InetAddressAndPort.getByName("[2001:0db8:0000:0000:0000:ff00:0042:8329]:42");
+        assertEquals(InetAddress.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329"), address.address);
+        assertEquals(42, address.port);
+    }
+
+    @Test
+    public void compareAndEqualsAndHashCodeTest() throws Exception
+    {
+        InetAddressAndPort address1 = InetAddressAndPort.getByName("127.0.0.1:42");
+        InetAddressAndPort address4 = InetAddressAndPort.getByName("127.0.0.1:43");
+        InetAddressAndPort address5 = InetAddressAndPort.getByName("127.0.0.1:41");
+        InetAddressAndPort address6 = InetAddressAndPort.getByName("127.0.0.2:42");
+        InetAddressAndPort address7 = InetAddressAndPort.getByName("127.0.0.0:42");
+
+        assertEquals(0, address1.compareTo(address1));
+        assertEquals(-1, address1.compareTo(address4));
+        assertEquals(1, address1.compareTo(address5));
+        assertEquals(-1, address1.compareTo(address6));
+        assertEquals(1, address1.compareTo(address7));
+
+        assertEquals(address1, address1);
+        assertEquals(address1.hashCode(), address1.hashCode());
+        assertEquals(address1, InetAddressAndPort.getByName("127.0.0.1:42"));
+        assertEquals(address1.hashCode(), InetAddressAndPort.getByName("127.0.0.1:42").hashCode());
+        assertEquals(address1, InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", 42));
+        assertEquals(address1.hashCode(), InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", 42).hashCode());
+        int originalPort = InetAddressAndPort.defaultPort;
+        InetAddressAndPort.initializeDefaultPort(42);
+        try
+        {
+            assertEquals(address1, InetAddressAndPort.getByName("127.0.0.1"));
+            assertEquals(address1.hashCode(), InetAddressAndPort.getByName("127.0.0.1").hashCode());
+        }
+        finally
+        {
+            InetAddressAndPort.initializeDefaultPort(originalPort);
+        }
+        assertTrue(!address1.equals(address4));
+        assertTrue(!address1.equals(address5));
+        assertTrue(!address1.equals(address6));
+        assertTrue(!address1.equals(address7));
+    }
+
+    @Test
+    public void toStringTest() throws Exception
+    {
+        String ipv4 = "127.0.0.1:42";
+        String ipv6 = "[2001:db8:0:0:0:ff00:42:8329]:42";
+        assertEquals(ipv4, InetAddressAndPort.getByName(ipv4).toString());
+        assertEquals(ipv6, InetAddressAndPort.getByName(ipv6).toString());
+    }
+
+
+    private void shouldThrow(ThrowingRunnable t, Class expectedClass)
+    {
+        try
+        {
+            t.run();
+        }
+        catch (Throwable thrown)
+        {
+            assertEquals(thrown.getClass(), expectedClass);
+            return;
+        }
+        fail("Runnable didn't throw");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
index f64b84a..ab6c6cd 100644
--- a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.locator;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 import java.util.stream.Collectors;
@@ -73,9 +72,9 @@ public class NetworkTopologyStrategyTest
         assert strategy.getReplicationFactor("DC2") == 2;
         assert strategy.getReplicationFactor("DC3") == 1;
         // Query for the natural hosts
-        ArrayList<InetAddress> endpoints = strategy.getNaturalEndpoints(new StringToken("123"));
+        ArrayList<InetAddressAndPort> endpoints = strategy.getNaturalEndpoints(new StringToken("123"));
         assert 6 == endpoints.size();
-        assert 6 == new HashSet<InetAddress>(endpoints).size(); // ensure uniqueness
+        assert 6 == new HashSet<>(endpoints).size(); // ensure uniqueness
     }
 
     @Test
@@ -97,9 +96,9 @@ public class NetworkTopologyStrategyTest
         assert strategy.getReplicationFactor("DC2") == 3;
         assert strategy.getReplicationFactor("DC3") == 0;
         // Query for the natural hosts
-        ArrayList<InetAddress> endpoints = strategy.getNaturalEndpoints(new StringToken("123"));
+        ArrayList<InetAddressAndPort> endpoints = strategy.getNaturalEndpoints(new StringToken("123"));
         assert 6 == endpoints.size();
-        assert 6 == new HashSet<InetAddress>(endpoints).size(); // ensure uniqueness
+        assert 6 == new HashSet<>(endpoints).size(); // ensure uniqueness
     }
 
     @Test
@@ -113,7 +112,7 @@ public class NetworkTopologyStrategyTest
         DatabaseDescriptor.setEndpointSnitch(snitch);
         TokenMetadata metadata = new TokenMetadata();
         Map<String, String> configOptions = new HashMap<String, String>();
-        Multimap<InetAddress, Token> tokens = HashMultimap.create();
+        Multimap<InetAddressAndPort, Token> tokens = HashMultimap.create();
 
         int totalRF = 0;
         for (int dc = 0; dc < dcRacks.length; ++dc)
@@ -125,7 +124,7 @@ public class NetworkTopologyStrategyTest
                 for (int ep = 1; ep <= dcEndpoints[dc]/dcRacks[dc]; ++ep)
                 {
                     byte[] ipBytes = new byte[]{10, (byte)dc, (byte)rack, (byte)ep};
-                    InetAddress address = InetAddress.getByAddress(ipBytes);
+                    InetAddressAndPort address = InetAddressAndPort.getByAddress(ipBytes);
                     StringToken token = new StringToken(String.format("%02x%02x%02x", ep, rack, dc));
                     logger.debug("adding node {} at {}", address, token);
                     tokens.put(address, token);
@@ -138,8 +137,8 @@ public class NetworkTopologyStrategyTest
 
         for (String testToken : new String[]{"123456", "200000", "000402", "ffffff", "400200"})
         {
-            List<InetAddress> endpoints = strategy.calculateNaturalEndpoints(new StringToken(testToken), metadata);
-            Set<InetAddress> epSet = new HashSet<InetAddress>(endpoints);
+            List<InetAddressAndPort> endpoints = strategy.calculateNaturalEndpoints(new StringToken(testToken), metadata);
+            Set<InetAddressAndPort> epSet = new HashSet<>(endpoints);
 
             Assert.assertEquals(totalRF, endpoints.size());
             Assert.assertEquals(totalRF, epSet.size());
@@ -172,7 +171,7 @@ public class NetworkTopologyStrategyTest
     public void tokenFactory(TokenMetadata metadata, String token, byte[] bytes) throws UnknownHostException
     {
         Token token1 = new StringToken(token);
-        InetAddress add1 = InetAddress.getByAddress(bytes);
+        InetAddressAndPort add1 = InetAddressAndPort.getByAddress(bytes);
         metadata.updateNormalToken(token1, add1);
     }
 
@@ -184,9 +183,9 @@ public class NetworkTopologyStrategyTest
         final int RUNS = 10;
         StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
         Map<String, Integer> datacenters = ImmutableMap.of("rf1", 1, "rf3", 3, "rf5_1", 5, "rf5_2", 5, "rf5_3", 5);
-        List<InetAddress> nodes = new ArrayList<>(NODES);
+        List<InetAddressAndPort> nodes = new ArrayList<>(NODES);
         for (byte i=0; i<NODES; ++i)
-            nodes.add(InetAddress.getByAddress(new byte[]{127, 0, 0, i}));
+            nodes.add(InetAddressAndPort.getByAddress(new byte[]{ 127, 0, 0, i}));
         for (int run=0; run<RUNS; ++run)
         {
             Random rand = new Random();
@@ -209,8 +208,8 @@ public class NetworkTopologyStrategyTest
         for (int i=0; i<1000; ++i)
         {
             Token token = Murmur3Partitioner.instance.getRandomToken(rand);
-            List<InetAddress> expected = calculateNaturalEndpoints(token, tokenMetadata, datacenters, snitch);
-            List<InetAddress> actual = nts.calculateNaturalEndpoints(token, tokenMetadata);
+            List<InetAddressAndPort> expected = calculateNaturalEndpoints(token, tokenMetadata, datacenters, snitch);
+            List<InetAddressAndPort> actual = nts.calculateNaturalEndpoints(token, tokenMetadata);
             if (endpointsDiffer(expected, actual))
             {
                 System.err.println("Endpoints mismatch for token " + token);
@@ -221,7 +220,7 @@ public class NetworkTopologyStrategyTest
         }
     }
 
-    private boolean endpointsDiffer(List<InetAddress> ep1, List<InetAddress> ep2)
+    private boolean endpointsDiffer(List<InetAddressAndPort> ep1, List<InetAddressAndPort> ep2)
     {
         // Because the old algorithm does not put the nodes in the correct order in the case where more replicas
         // are required than there are racks in a dc, we accept different order as long as the primary
@@ -230,15 +229,15 @@ public class NetworkTopologyStrategyTest
             return false;
         if (!ep1.get(0).equals(ep2.get(0)))
             return true;
-        Set<InetAddress> s1 = new HashSet<>(ep1);
-        Set<InetAddress> s2 = new HashSet<>(ep2);
+        Set<InetAddressAndPort> s1 = new HashSet<>(ep1);
+        Set<InetAddressAndPort> s2 = new HashSet<>(ep2);
         return !s1.equals(s2);
     }
 
-    IEndpointSnitch generateSnitch(Map<String, Integer> datacenters, Collection<InetAddress> nodes, Random rand)
+    IEndpointSnitch generateSnitch(Map<String, Integer> datacenters, Collection<InetAddressAndPort> nodes, Random rand)
     {
-        final Map<InetAddress, String> nodeToRack = new HashMap<>();
-        final Map<InetAddress, String> nodeToDC = new HashMap<>();
+        final Map<InetAddressAndPort, String> nodeToRack = new HashMap<>();
+        final Map<InetAddressAndPort, String> nodeToDC = new HashMap<>();
         Map<String, List<String>> racksPerDC = new HashMap<>();
         datacenters.forEach((dc, rf) -> racksPerDC.put(dc, randomRacks(rf, rand)));
         int rf = datacenters.values().stream().mapToInt(x -> x).sum();
@@ -250,7 +249,7 @@ public class NetworkTopologyStrategyTest
                 dcs[pos++] = dce.getKey();
         }
 
-        for (InetAddress node : nodes)
+        for (InetAddressAndPort node : nodes)
         {
             String dc = dcs[rand.nextInt(rf)];
             List<String> racks = racksPerDC.get(dc);
@@ -261,12 +260,12 @@ public class NetworkTopologyStrategyTest
 
         return new AbstractNetworkTopologySnitch()
         {
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 return nodeToRack.get(endpoint);
             }
 
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
                 return nodeToDC.get(endpoint);
             }
@@ -283,20 +282,20 @@ public class NetworkTopologyStrategyTest
     }
 
     // Copy of older endpoints calculation algorithm for comparison
-    public static List<InetAddress> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata, Map<String, Integer> datacenters, IEndpointSnitch snitch)
+    public static List<InetAddressAndPort> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata, Map<String, Integer> datacenters, IEndpointSnitch snitch)
     {
         // we want to preserve insertion order so that the first added endpoint becomes primary
-        Set<InetAddress> replicas = new LinkedHashSet<>();
+        Set<InetAddressAndPort> replicas = new LinkedHashSet<>();
         // replicas we have found in each DC
-        Map<String, Set<InetAddress>> dcReplicas = new HashMap<>(datacenters.size());
+        Map<String, Set<InetAddressAndPort>> dcReplicas = new HashMap<>(datacenters.size());
         for (Map.Entry<String, Integer> dc : datacenters.entrySet())
-            dcReplicas.put(dc.getKey(), new HashSet<InetAddress>(dc.getValue()));
+            dcReplicas.put(dc.getKey(), new HashSet<InetAddressAndPort>(dc.getValue()));
 
         Topology topology = tokenMetadata.getTopology();
         // all endpoints in each DC, so we can check when we have exhausted all the members of a DC
-        Multimap<String, InetAddress> allEndpoints = topology.getDatacenterEndpoints();
+        Multimap<String, InetAddressAndPort> allEndpoints = topology.getDatacenterEndpoints();
         // all racks in a DC so we can check when we have exhausted all racks in a DC
-        Map<String, Multimap<String, InetAddress>> racks = topology.getDatacenterRacks();
+        Map<String, Multimap<String, InetAddressAndPort>> racks = topology.getDatacenterRacks();
         assert !allEndpoints.isEmpty() && !racks.isEmpty() : "not aware of any cluster members";
 
         // tracks the racks we have already placed replicas in
@@ -306,15 +305,15 @@ public class NetworkTopologyStrategyTest
 
         // tracks the endpoints that we skipped over while looking for unique racks
         // when we relax the rack uniqueness we can append this to the current result so we don't have to wind back the iterator
-        Map<String, Set<InetAddress>> skippedDcEndpoints = new HashMap<>(datacenters.size());
+        Map<String, Set<InetAddressAndPort>> skippedDcEndpoints = new HashMap<>(datacenters.size());
         for (Map.Entry<String, Integer> dc : datacenters.entrySet())
-            skippedDcEndpoints.put(dc.getKey(), new LinkedHashSet<InetAddress>());
+            skippedDcEndpoints.put(dc.getKey(), new LinkedHashSet<InetAddressAndPort>());
 
         Iterator<Token> tokenIter = TokenMetadata.ringIterator(tokenMetadata.sortedTokens(), searchToken, false);
         while (tokenIter.hasNext() && !hasSufficientReplicas(dcReplicas, allEndpoints, datacenters))
         {
             Token next = tokenIter.next();
-            InetAddress ep = tokenMetadata.getEndpoint(next);
+            InetAddressAndPort ep = tokenMetadata.getEndpoint(next);
             String dc = snitch.getDatacenter(ep);
             // have we already found all replicas for this dc?
             if (!datacenters.containsKey(dc) || hasSufficientReplicas(dc, dcReplicas, allEndpoints, datacenters))
@@ -341,10 +340,10 @@ public class NetworkTopologyStrategyTest
                     // if we've run out of distinct racks, add the hosts we skipped past already (up to RF)
                     if (seenRacks.get(dc).size() == racks.get(dc).keySet().size())
                     {
-                        Iterator<InetAddress> skippedIt = skippedDcEndpoints.get(dc).iterator();
+                        Iterator<InetAddressAndPort> skippedIt = skippedDcEndpoints.get(dc).iterator();
                         while (skippedIt.hasNext() && !hasSufficientReplicas(dc, dcReplicas, allEndpoints, datacenters))
                         {
-                            InetAddress nextSkipped = skippedIt.next();
+                            InetAddressAndPort nextSkipped = skippedIt.next();
                             dcReplicas.get(dc).add(nextSkipped);
                             replicas.add(nextSkipped);
                         }
@@ -353,15 +352,15 @@ public class NetworkTopologyStrategyTest
             }
         }
 
-        return new ArrayList<InetAddress>(replicas);
+        return new ArrayList<InetAddressAndPort>(replicas);
     }
 
-    private static boolean hasSufficientReplicas(String dc, Map<String, Set<InetAddress>> dcReplicas, Multimap<String, InetAddress> allEndpoints, Map<String, Integer> datacenters)
+    private static boolean hasSufficientReplicas(String dc, Map<String, Set<InetAddressAndPort>> dcReplicas, Multimap<String, InetAddressAndPort> allEndpoints, Map<String, Integer> datacenters)
     {
         return dcReplicas.get(dc).size() >= Math.min(allEndpoints.get(dc).size(), getReplicationFactor(dc, datacenters));
     }
 
-    private static boolean hasSufficientReplicas(Map<String, Set<InetAddress>> dcReplicas, Multimap<String, InetAddress> allEndpoints, Map<String, Integer> datacenters)
+    private static boolean hasSufficientReplicas(Map<String, Set<InetAddressAndPort>> dcReplicas, Multimap<String, InetAddressAndPort> allEndpoints, Map<String, Integer> datacenters)
     {
         for (String dc : datacenters.keySet())
             if (!hasSufficientReplicas(dc, dcReplicas, allEndpoints, datacenters))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
index e6e17cd..9c90d57 100644
--- a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -45,7 +44,7 @@ public class OldNetworkTopologyStrategyTest
 {
     private List<Token> keyTokens;
     private TokenMetadata tmd;
-    private Map<String, ArrayList<InetAddress>> expectedResults;
+    private Map<String, ArrayList<InetAddressAndPort>> expectedResults;
 
     @BeforeClass
     public static void setupDD()
@@ -58,7 +57,7 @@ public class OldNetworkTopologyStrategyTest
     {
         keyTokens = new ArrayList<Token>();
         tmd = new TokenMetadata();
-        expectedResults = new HashMap<String, ArrayList<InetAddress>>();
+        expectedResults = new HashMap<String, ArrayList<InetAddressAndPort>>();
     }
 
     /**
@@ -136,12 +135,12 @@ public class OldNetworkTopologyStrategyTest
         testGetEndpoints(strategy, keyTokens.toArray(new Token[0]));
     }
 
-    private ArrayList<InetAddress> buildResult(String... addresses) throws UnknownHostException
+    private ArrayList<InetAddressAndPort> buildResult(String... addresses) throws UnknownHostException
     {
-        ArrayList<InetAddress> result = new ArrayList<InetAddress>();
+        ArrayList<InetAddressAndPort> result = new ArrayList<>();
         for (String address : addresses)
         {
-            result.add(InetAddress.getByName(address));
+            result.add(InetAddressAndPort.getByName(address));
         }
         return result;
     }
@@ -153,7 +152,7 @@ public class OldNetworkTopologyStrategyTest
         BigIntegerToken keyToken = new BigIntegerToken(keyTokenID);
         keyTokens.add(keyToken);
 
-        InetAddress ep = InetAddress.getByName(endpointAddress);
+        InetAddressAndPort ep = InetAddressAndPort.getByName(endpointAddress);
         tmd.updateNormalToken(endpointToken, ep);
     }
 
@@ -161,10 +160,10 @@ public class OldNetworkTopologyStrategyTest
     {
         for (Token keyToken : keyTokens)
         {
-            List<InetAddress> endpoints = strategy.getNaturalEndpoints(keyToken);
+            List<InetAddressAndPort> endpoints = strategy.getNaturalEndpoints(keyToken);
             for (int j = 0; j < endpoints.size(); j++)
             {
-                ArrayList<InetAddress> hostsExpected = expectedResults.get(keyToken.toString());
+                ArrayList<InetAddressAndPort> hostsExpected = expectedResults.get(keyToken.toString());
                 assertEquals(endpoints.get(j), hostsExpected.get(j));
             }
         }
@@ -340,7 +339,7 @@ public class OldNetworkTopologyStrategyTest
 
         int lastIPPart = 1;
         for (BigIntegerToken token : tokens)
-            tokenMetadataCurrent.updateNormalToken(token, InetAddress.getByName("254.0.0." + Integer.toString(lastIPPart++)));
+            tokenMetadataCurrent.updateNormalToken(token, InetAddressAndPort.getByName("254.0.0." + Integer.toString(lastIPPart++)));
 
         return tokenMetadataCurrent;
     }
@@ -360,7 +359,7 @@ public class OldNetworkTopologyStrategyTest
     {
         RackInferringSnitch endpointSnitch = new RackInferringSnitch();
 
-        InetAddress movingNode = InetAddress.getByName("254.0.0." + Integer.toString(movingNodeIdx + 1));
+        InetAddressAndPort movingNode = InetAddressAndPort.getByName("254.0.0." + Integer.toString(movingNodeIdx + 1));
 
 
         TokenMetadata tokenMetadataCurrent = initTokenMetadata(tokens);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java b/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
index 7121550..56fd181 100644
--- a/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
+++ b/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
@@ -25,7 +25,6 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.junit.Test;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
 
@@ -44,12 +43,12 @@ public class PendingRangeMapsTest {
     {
         PendingRangeMaps pendingRangeMaps = new PendingRangeMaps();
 
-        pendingRangeMaps.addPendingRange(genRange("5", "15"), InetAddress.getByName("127.0.0.1"));
-        pendingRangeMaps.addPendingRange(genRange("15", "25"), InetAddress.getByName("127.0.0.2"));
-        pendingRangeMaps.addPendingRange(genRange("25", "35"), InetAddress.getByName("127.0.0.3"));
-        pendingRangeMaps.addPendingRange(genRange("35", "45"), InetAddress.getByName("127.0.0.4"));
-        pendingRangeMaps.addPendingRange(genRange("45", "55"), InetAddress.getByName("127.0.0.5"));
-        pendingRangeMaps.addPendingRange(genRange("45", "65"), InetAddress.getByName("127.0.0.6"));
+        pendingRangeMaps.addPendingRange(genRange("5", "15"), InetAddressAndPort.getByName("127.0.0.1"));
+        pendingRangeMaps.addPendingRange(genRange("15", "25"), InetAddressAndPort.getByName("127.0.0.2"));
+        pendingRangeMaps.addPendingRange(genRange("25", "35"), InetAddressAndPort.getByName("127.0.0.3"));
+        pendingRangeMaps.addPendingRange(genRange("35", "45"), InetAddressAndPort.getByName("127.0.0.4"));
+        pendingRangeMaps.addPendingRange(genRange("45", "55"), InetAddressAndPort.getByName("127.0.0.5"));
+        pendingRangeMaps.addPendingRange(genRange("45", "65"), InetAddressAndPort.getByName("127.0.0.6"));
 
         assertEquals(0, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("0")).size());
         assertEquals(0, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("5")).size());
@@ -62,8 +61,8 @@ public class PendingRangeMapsTest {
         assertEquals(2, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("55")).size());
         assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("65")).size());
 
-        Collection<InetAddress> endpoints = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("15"));
-        assertTrue(endpoints.contains(InetAddress.getByName("127.0.0.1")));
+        Collection<InetAddressAndPort> endpoints = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("15"));
+        assertTrue(endpoints.contains(InetAddressAndPort.getByName("127.0.0.1")));
     }
 
     @Test
@@ -71,13 +70,13 @@ public class PendingRangeMapsTest {
     {
         PendingRangeMaps pendingRangeMaps = new PendingRangeMaps();
 
-        pendingRangeMaps.addPendingRange(genRange("5", "15"), InetAddress.getByName("127.0.0.1"));
-        pendingRangeMaps.addPendingRange(genRange("15", "25"), InetAddress.getByName("127.0.0.2"));
-        pendingRangeMaps.addPendingRange(genRange("25", "35"), InetAddress.getByName("127.0.0.3"));
-        pendingRangeMaps.addPendingRange(genRange("35", "45"), InetAddress.getByName("127.0.0.4"));
-        pendingRangeMaps.addPendingRange(genRange("45", "55"), InetAddress.getByName("127.0.0.5"));
-        pendingRangeMaps.addPendingRange(genRange("45", "65"), InetAddress.getByName("127.0.0.6"));
-        pendingRangeMaps.addPendingRange(genRange("65", "7"), InetAddress.getByName("127.0.0.7"));
+        pendingRangeMaps.addPendingRange(genRange("5", "15"), InetAddressAndPort.getByName("127.0.0.1"));
+        pendingRangeMaps.addPendingRange(genRange("15", "25"), InetAddressAndPort.getByName("127.0.0.2"));
+        pendingRangeMaps.addPendingRange(genRange("25", "35"), InetAddressAndPort.getByName("127.0.0.3"));
+        pendingRangeMaps.addPendingRange(genRange("35", "45"), InetAddressAndPort.getByName("127.0.0.4"));
+        pendingRangeMaps.addPendingRange(genRange("45", "55"), InetAddressAndPort.getByName("127.0.0.5"));
+        pendingRangeMaps.addPendingRange(genRange("45", "65"), InetAddressAndPort.getByName("127.0.0.6"));
+        pendingRangeMaps.addPendingRange(genRange("65", "7"), InetAddressAndPort.getByName("127.0.0.7"));
 
         assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("0")).size());
         assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("5")).size());
@@ -91,8 +90,8 @@ public class PendingRangeMapsTest {
         assertEquals(2, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("55")).size());
         assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("65")).size());
 
-        Collection<InetAddress> endpoints = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("6"));
-        assertTrue(endpoints.contains(InetAddress.getByName("127.0.0.1")));
-        assertTrue(endpoints.contains(InetAddress.getByName("127.0.0.7")));
+        Collection<InetAddressAndPort> endpoints = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("6"));
+        assertTrue(endpoints.contains(InetAddressAndPort.getByName("127.0.0.1")));
+        assertTrue(endpoints.contains(InetAddressAndPort.getByName("127.0.0.7")));
     }
 }


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


[16/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
index 8c0ed1e..f01197d 100644
--- a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
@@ -52,11 +52,13 @@ public class ConfigHelper
     private static final int DEFAULT_RANGE_BATCH_SIZE = 4096;
     private static final String INPUT_INITIAL_ADDRESS = "cassandra.input.address";
     private static final String OUTPUT_INITIAL_ADDRESS = "cassandra.output.address";
+    private static final String OUTPUT_INITIAL_PORT = "cassandra.output.port";
     private static final String READ_CONSISTENCY_LEVEL = "cassandra.consistencylevel.read";
     private static final String WRITE_CONSISTENCY_LEVEL = "cassandra.consistencylevel.write";
     private static final String OUTPUT_COMPRESSION_CLASS = "cassandra.output.compression.class";
     private static final String OUTPUT_COMPRESSION_CHUNK_LENGTH = "cassandra.output.compression.length";
     private static final String OUTPUT_LOCAL_DC_ONLY = "cassandra.output.local.dc.only";
+    private static final String DEFAULT_CASSANDRA_NATIVE_PORT = "7000";
 
     private static final Logger logger = LoggerFactory.getLogger(ConfigHelper.class);
 
@@ -349,6 +351,16 @@ public class ConfigHelper
         return conf.get(OUTPUT_INITIAL_ADDRESS);
     }
 
+    public static void setOutputInitialPort(Configuration conf, Integer port)
+    {
+        conf.set(OUTPUT_INITIAL_PORT, port.toString());
+    }
+
+    public static Integer getOutputInitialPort(Configuration conf)
+    {
+        return Integer.valueOf(conf.get(OUTPUT_INITIAL_PORT, DEFAULT_CASSANDRA_NATIVE_PORT));
+    }
+
     public static void setOutputInitialAddress(Configuration conf, String address)
     {
         conf.set(OUTPUT_INITIAL_ADDRESS, address);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
index 0f44e0c..204d9ee 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
@@ -21,11 +21,13 @@ import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
 
+import com.google.common.net.HostAndPort;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,6 +41,7 @@ import org.apache.cassandra.hadoop.HadoopCompat;
 import org.apache.cassandra.io.sstable.CQLSSTableWriter;
 import org.apache.cassandra.io.sstable.SSTableLoader;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.utils.NativeSSTableLoaderClient;
@@ -80,7 +83,7 @@ public class CqlBulkRecordWriter extends RecordWriter<Object, List<ByteBuffer>>
     protected SSTableLoader loader;
     protected Progressable progress;
     protected TaskAttemptContext context;
-    protected final Set<InetAddress> ignores = new HashSet<>();
+    protected final Set<InetAddressAndPort> ignores = new HashSet<>();
 
     private String keyspace;
     private String table;
@@ -139,7 +142,7 @@ public class CqlBulkRecordWriter extends RecordWriter<Object, List<ByteBuffer>>
         try
         {
             for (String hostToIgnore : CqlBulkOutputFormat.getIgnoreHosts(conf))
-                ignores.add(InetAddress.getByName(hostToIgnore));
+                ignores.add(InetAddressAndPort.getByName(hostToIgnore));
         }
         catch (UnknownHostException e)
         {
@@ -285,20 +288,23 @@ public class CqlBulkRecordWriter extends RecordWriter<Object, List<ByteBuffer>>
         {
             super(resolveHostAddresses(conf),
                   CqlConfigHelper.getOutputNativePort(conf),
+                  ConfigHelper.getOutputInitialPort(conf),
                   ConfigHelper.getOutputKeyspaceUserName(conf),
                   ConfigHelper.getOutputKeyspacePassword(conf),
-                  CqlConfigHelper.getSSLOptions(conf).orNull());
+                  CqlConfigHelper.getSSLOptions(conf).orNull(),
+                  CqlConfigHelper.getAllowServerPortDiscovery(conf));
         }
 
-        private static Collection<InetAddress> resolveHostAddresses(Configuration conf)
+        private static Collection<InetSocketAddress> resolveHostAddresses(Configuration conf)
         {
-            Set<InetAddress> addresses = new HashSet<>();
-
+            Set<InetSocketAddress> addresses = new HashSet<>();
+            int port = CqlConfigHelper.getOutputNativePort(conf);
             for (String host : ConfigHelper.getOutputInitialAddress(conf).split(","))
             {
                 try
                 {
-                    addresses.add(InetAddress.getByName(host));
+                    HostAndPort hap = HostAndPort.fromString(host);
+                    addresses.add(new InetSocketAddress(InetAddress.getByName(hap.getHost()), hap.getPortOrDefault(port)));
                 }
                 catch (UnknownHostException e)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java
index f9a6f3a..3a47a72 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java
@@ -88,6 +88,7 @@ public class CqlConfigHelper
 
     private static final String OUTPUT_CQL = "cassandra.output.cql";
     private static final String OUTPUT_NATIVE_PORT = "cassandra.output.native.port";
+    private static final String ALLOW_SERVER_PORT_DISCOVERY = "cassandra.allowserverportdiscovery";
 
     /**
      * Set the CQL columns for the input of this job.
@@ -651,4 +652,15 @@ public class CqlConfigHelper
                  new SecureRandom());
         return ctx;
     }
+
+    public static void setAllowServerPortDiscovery(Configuration conf, boolean allowServerPortDiscovery)
+    {
+        conf.set(ALLOW_SERVER_PORT_DISCOVERY, Boolean.toString(allowServerPortDiscovery));
+    }
+
+    public static boolean getAllowServerPortDiscovery(Configuration conf)
+    {
+        return Boolean.parseBoolean(conf.get(ALLOW_SERVER_PORT_DISCOVERY, "false"));
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hints/HintVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintVerbHandler.java b/src/java/org/apache/cassandra/hints/HintVerbHandler.java
index 2b92a42..cec6f0b 100644
--- a/src/java/org/apache/cassandra/hints/HintVerbHandler.java
+++ b/src/java/org/apache/cassandra/hints/HintVerbHandler.java
@@ -18,13 +18,13 @@
  */
 package org.apache.cassandra.hints;
 
-import java.net.InetAddress;
 import java.util.UUID;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
@@ -47,7 +47,7 @@ public final class HintVerbHandler implements IVerbHandler<HintMessage>
     {
         UUID hostId = message.payload.hostId;
         Hint hint = message.payload.hint;
-        InetAddress address = StorageService.instance.getEndpointForHostId(hostId);
+        InetAddressAndPort address = StorageService.instance.getEndpointForHostId(hostId);
 
         // If we see an unknown table id, it means the table, or one of the tables in the mutation, had been dropped.
         // In that case there is nothing we can really do, or should do, other than log it go on.
@@ -96,7 +96,7 @@ public final class HintVerbHandler implements IVerbHandler<HintMessage>
         }
     }
 
-    private static void reply(int id, InetAddress to)
+    private static void reply(int id, InetAddressAndPort to)
     {
         MessagingService.instance().sendReply(HintResponse.message, id, to);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
index 58a3e6f..cbbb212 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.hints;
 
 import java.io.File;
-import java.net.InetAddress;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.*;
@@ -36,6 +35,7 @@ import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageService;
 
 /**
@@ -50,10 +50,10 @@ final class HintsDispatchExecutor
     private final File hintsDirectory;
     private final ExecutorService executor;
     private final AtomicBoolean isPaused;
-    private final Predicate<InetAddress> isAlive;
+    private final Predicate<InetAddressAndPort> isAlive;
     private final Map<UUID, Future> scheduledDispatches;
 
-    HintsDispatchExecutor(File hintsDirectory, int maxThreads, AtomicBoolean isPaused, Predicate<InetAddress> isAlive)
+    HintsDispatchExecutor(File hintsDirectory, int maxThreads, AtomicBoolean isPaused, Predicate<InetAddressAndPort> isAlive)
     {
         this.hintsDirectory = hintsDirectory;
         this.isPaused = isPaused;
@@ -154,7 +154,7 @@ final class HintsDispatchExecutor
         public void run()
         {
             UUID hostId = hostIdSupplier.get();
-            InetAddress address = StorageService.instance.getEndpointForHostId(hostId);
+            InetAddressAndPort address = StorageService.instance.getEndpointForHostId(hostId);
             logger.info("Transferring all hints to {}: {}", address, hostId);
             if (transfer(hostId))
                 return;
@@ -257,7 +257,7 @@ final class HintsDispatchExecutor
         {
             logger.trace("Dispatching hints file {}", descriptor.fileName());
 
-            InetAddress address = StorageService.instance.getEndpointForHostId(hostId);
+            InetAddressAndPort address = StorageService.instance.getEndpointForHostId(hostId);
             if (address != null)
                 return deliver(descriptor, address);
 
@@ -266,7 +266,7 @@ final class HintsDispatchExecutor
             return true;
         }
 
-        private boolean deliver(HintsDescriptor descriptor, InetAddress address)
+        private boolean deliver(HintsDescriptor descriptor, InetAddressAndPort address)
         {
             File file = new File(hintsDirectory, descriptor.fileName());
             InputPosition offset = store.getDispatchOffset(descriptor);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java b/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
index 34d1eb2..ca38c0c 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
@@ -23,7 +23,7 @@ import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.schema.Schema;
 
-import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddress;
+import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort;
 
 /**
  * A simple dispatch trigger that's being run every 10 seconds.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hints/HintsDispatcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatcher.java b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
index 323eeb1..d0d9aac 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatcher.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.hints;
 
 import java.io.File;
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
@@ -31,6 +30,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.monitoring.ApproximateTime;
 import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.HintsServiceMetrics;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
@@ -51,13 +51,13 @@ final class HintsDispatcher implements AutoCloseable
 
     private final HintsReader reader;
     private final UUID hostId;
-    private final InetAddress address;
+    private final InetAddressAndPort address;
     private final int messagingVersion;
     private final BooleanSupplier abortRequested;
 
     private InputPosition currentPagePosition;
 
-    private HintsDispatcher(HintsReader reader, UUID hostId, InetAddress address, int messagingVersion, BooleanSupplier abortRequested)
+    private HintsDispatcher(HintsReader reader, UUID hostId, InetAddressAndPort address, int messagingVersion, BooleanSupplier abortRequested)
     {
         currentPagePosition = null;
 
@@ -68,7 +68,7 @@ final class HintsDispatcher implements AutoCloseable
         this.abortRequested = abortRequested;
     }
 
-    static HintsDispatcher create(File file, RateLimiter rateLimiter, InetAddress address, UUID hostId, BooleanSupplier abortRequested)
+    static HintsDispatcher create(File file, RateLimiter rateLimiter, InetAddressAndPort address, UUID hostId, BooleanSupplier abortRequested)
     {
         int messagingVersion = MessagingService.instance().getVersion(address);
         return new HintsDispatcher(HintsReader.open(file, rateLimiter), hostId, address, messagingVersion, abortRequested);
@@ -228,7 +228,7 @@ final class HintsDispatcher implements AutoCloseable
             return timedOut ? Outcome.TIMEOUT : outcome;
         }
 
-        public void onFailure(InetAddress from, RequestFailureReason failureReason)
+        public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
         {
             outcome = Outcome.FAILURE;
             condition.signalAll();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hints/HintsService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsService.java b/src/java/org/apache/cassandra/hints/HintsService.java
index 3d82c02..5c331d0 100644
--- a/src/java/org/apache/cassandra/hints/HintsService.java
+++ b/src/java/org/apache/cassandra/hints/HintsService.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.hints;
 
 import java.io.File;
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collections;
 import java.util.UUID;
@@ -40,6 +39,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.gms.IFailureDetector;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.HintedHandoffMetrics;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.dht.Token;
@@ -267,10 +267,10 @@ public final class HintsService implements HintsServiceMBean
      */
     public void deleteAllHintsForEndpoint(String address)
     {
-        InetAddress target;
+        InetAddressAndPort target;
         try
         {
-            target = InetAddress.getByName(address);
+            target = InetAddressAndPort.getByName(address);
         }
         catch (UnknownHostException e)
         {
@@ -284,7 +284,7 @@ public final class HintsService implements HintsServiceMBean
      *
      * @param target inet address of the target node
      */
-    public void deleteAllHintsForEndpoint(InetAddress target)
+    public void deleteAllHintsForEndpoint(InetAddressAndPort target)
     {
         UUID hostId = StorageService.instance.getHostIdForEndpoint(target);
         if (hostId == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/hints/HintsStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsStore.java b/src/java/org/apache/cassandra/hints/HintsStore.java
index 3572172..bbf57f5 100644
--- a/src/java/org/apache/cassandra/hints/HintsStore.java
+++ b/src/java/org/apache/cassandra/hints/HintsStore.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.hints;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedDeque;
@@ -31,6 +30,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.SyncUtil;
 
@@ -77,14 +77,14 @@ final class HintsStore
         return new HintsStore(hostId, hintsDirectory, writerParams, descriptors);
     }
 
-    InetAddress address()
+    InetAddressAndPort address()
     {
         return StorageService.instance.getEndpointForHostId(hostId);
     }
 
     boolean isLive()
     {
-        InetAddress address = address();
+        InetAddressAndPort address = address();
         return address != null && FailureDetector.instance.isAlive(address);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java b/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java
new file mode 100644
index 0000000..d82ff7d
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io;
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+
+/**
+ * Serializes a dummy byte that can't be set. Will always write 0 and return 0 in a correctly formed message.
+ */
+public class DummyByteVersionedSerializer implements IVersionedSerializer<byte[]>
+{
+    public static final DummyByteVersionedSerializer instance = new DummyByteVersionedSerializer();
+
+    private DummyByteVersionedSerializer() {}
+
+    public void serialize(byte[] bytes, DataOutputPlus out, int version) throws IOException
+    {
+        Preconditions.checkArgument(bytes == MessagingService.ONE_BYTE);
+        out.write(0);
+    }
+
+    public byte[] deserialize(DataInputPlus in, int version) throws IOException
+    {
+        assert(0 == in.readByte());
+        return MessagingService.ONE_BYTE;
+    }
+
+    public long serializedSize(byte[] bytes, int version)
+    {
+        //Payload
+        return 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/io/ShortVersionedSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/ShortVersionedSerializer.java b/src/java/org/apache/cassandra/io/ShortVersionedSerializer.java
new file mode 100644
index 0000000..8731f4c
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/ShortVersionedSerializer.java
@@ -0,0 +1,47 @@
+/*
+ * 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.io;
+
+import java.io.IOException;
+
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+public class ShortVersionedSerializer implements IVersionedSerializer<Short>
+{
+
+    public static final ShortVersionedSerializer instance = new ShortVersionedSerializer();
+
+    private ShortVersionedSerializer() {}
+
+    public void serialize(Short aShort, DataOutputPlus out, int version) throws IOException
+    {
+        out.writeShort(aShort);
+    }
+
+    public Short deserialize(DataInputPlus in, int version) throws IOException
+    {
+        return in.readShort();
+    }
+
+    public long serializedSize(Short aShort, int version)
+    {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 9fb3059..7d77ad5 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -19,12 +19,12 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.*;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.db.Directories;
@@ -32,7 +32,6 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.streaming.*;
 import org.apache.cassandra.utils.OutputHandler;
 import org.apache.cassandra.utils.Pair;
@@ -50,10 +49,10 @@ public class SSTableLoader implements StreamEventHandler
     private final Client client;
     private final int connectionsPerHost;
     private final OutputHandler outputHandler;
-    private final Set<InetAddress> failedHosts = new HashSet<>();
+    private final Set<InetAddressAndPort> failedHosts = new HashSet<>();
 
     private final List<SSTableReader> sstables = new ArrayList<>();
-    private final Multimap<InetAddress, StreamSession.SSTableStreamingSections> streamingDetails = HashMultimap.create();
+    private final Multimap<InetAddressAndPort, StreamSession.SSTableStreamingSections> streamingDetails = HashMultimap.create();
 
     public SSTableLoader(File directory, Client client, OutputHandler outputHandler)
     {
@@ -70,7 +69,7 @@ public class SSTableLoader implements StreamEventHandler
     }
 
     @SuppressWarnings("resource")
-    protected Collection<SSTableReader> openSSTables(final Map<InetAddress, Collection<Range<Token>>> ranges)
+    protected Collection<SSTableReader> openSSTables(final Map<InetAddressAndPort, Collection<Range<Token>>> ranges)
     {
         outputHandler.output("Opening sstables and calculating sections to stream");
 
@@ -124,9 +123,9 @@ public class SSTableLoader implements StreamEventHandler
 
                                               // calculate the sstable sections to stream as well as the estimated number of
                                               // keys per host
-                                              for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : ranges.entrySet())
+                                              for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : ranges.entrySet())
                                               {
-                                                  InetAddress endpoint = entry.getKey();
+                                                  InetAddressAndPort endpoint = entry.getKey();
                                                   Collection<Range<Token>> tokenRanges = entry.getValue();
 
                                                   List<Pair<Long, Long>> sstableSections = sstable.getPositionsForRanges(tokenRanges);
@@ -153,17 +152,17 @@ public class SSTableLoader implements StreamEventHandler
 
     public StreamResultFuture stream()
     {
-        return stream(Collections.<InetAddress>emptySet());
+        return stream(Collections.<InetAddressAndPort>emptySet());
     }
 
-    public StreamResultFuture stream(Set<InetAddress> toIgnore, StreamEventHandler... listeners)
+    public StreamResultFuture stream(Set<InetAddressAndPort> toIgnore, StreamEventHandler... listeners)
     {
         client.init(keyspace);
         outputHandler.output("Established connection to initial hosts");
 
         StreamPlan plan = new StreamPlan(StreamOperation.BULK_LOAD, connectionsPerHost, false, false, null, PreviewKind.NONE).connectionFactory(client.getConnectionFactory());
 
-        Map<InetAddress, Collection<Range<Token>>> endpointToRanges = client.getEndpointToRangesMap();
+        Map<InetAddressAndPort, Collection<Range<Token>>> endpointToRanges = client.getEndpointToRangesMap();
         openSSTables(endpointToRanges);
         if (sstables.isEmpty())
         {
@@ -173,9 +172,9 @@ public class SSTableLoader implements StreamEventHandler
 
         outputHandler.output(String.format("Streaming relevant part of %s to %s", names(sstables), endpointToRanges.keySet()));
 
-        for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : endpointToRanges.entrySet())
+        for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : endpointToRanges.entrySet())
         {
-            InetAddress remote = entry.getKey();
+            InetAddressAndPort remote = entry.getKey();
             if (toIgnore.contains(remote))
                 continue;
 
@@ -232,14 +231,14 @@ public class SSTableLoader implements StreamEventHandler
         return builder.toString();
     }
 
-    public Set<InetAddress> getFailedHosts()
+    public Set<InetAddressAndPort> getFailedHosts()
     {
         return failedHosts;
     }
 
     public static abstract class Client
     {
-        private final Map<InetAddress, Collection<Range<Token>>> endpointToRanges = new HashMap<>();
+        private final Map<InetAddressAndPort, Collection<Range<Token>>> endpointToRanges = new HashMap<>();
 
         /**
          * Initialize the client.
@@ -281,12 +280,12 @@ public class SSTableLoader implements StreamEventHandler
             throw new RuntimeException();
         }
 
-        public Map<InetAddress, Collection<Range<Token>>> getEndpointToRangesMap()
+        public Map<InetAddressAndPort, Collection<Range<Token>>> getEndpointToRangesMap()
         {
             return endpointToRanges;
         }
 
-        protected void addRangeForEndpoint(Range<Token> range, InetAddress endpoint)
+        protected void addRangeForEndpoint(Range<Token> range, InetAddressAndPort endpoint)
         {
             Collection<Range<Token>> ranges = endpointToRanges.get(endpoint);
             if (ranges == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java b/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
index 546d15e..2ee8eea 100644
--- a/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
@@ -17,14 +17,13 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.*;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 
 public abstract class AbstractEndpointSnitch implements IEndpointSnitch
 {
-    public abstract int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2);
+    public abstract int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2);
 
     /**
      * Sorts the <tt>Collection</tt> of node addresses by proximity to the given address
@@ -32,9 +31,9 @@ public abstract class AbstractEndpointSnitch implements IEndpointSnitch
      * @param unsortedAddress the nodes to sort
      * @return a new sorted <tt>List</tt>
      */
-    public List<InetAddress> getSortedListByProximity(InetAddress address, Collection<InetAddress> unsortedAddress)
+    public List<InetAddressAndPort> getSortedListByProximity(InetAddressAndPort address, Collection<InetAddressAndPort> unsortedAddress)
     {
-        List<InetAddress> preferred = new ArrayList<InetAddress>(unsortedAddress);
+        List<InetAddressAndPort> preferred = new ArrayList<>(unsortedAddress);
         sortByProximity(address, preferred);
         return preferred;
     }
@@ -44,11 +43,11 @@ public abstract class AbstractEndpointSnitch implements IEndpointSnitch
      * @param address the address to sort the proximity by
      * @param addresses the nodes to sort
      */
-    public void sortByProximity(final InetAddress address, List<InetAddress> addresses)
+    public void sortByProximity(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
     {
-        Collections.sort(addresses, new Comparator<InetAddress>()
+        Collections.sort(addresses, new Comparator<InetAddressAndPort>()
         {
-            public int compare(InetAddress a1, InetAddress a2)
+            public int compare(InetAddressAndPort a1, InetAddressAndPort a2)
             {
                 return compareEndpoints(address, a1, a2);
             }
@@ -60,7 +59,7 @@ public abstract class AbstractEndpointSnitch implements IEndpointSnitch
         // noop by default
     }
 
-    public boolean isWorthMergingForRangeQuery(List<InetAddress> merged, List<InetAddress> l1, List<InetAddress> l2)
+    public boolean isWorthMergingForRangeQuery(List<InetAddressAndPort> merged, List<InetAddressAndPort> l1, List<InetAddressAndPort> l2)
     {
         // Querying remote DC is likely to be an order of magnitude slower than
         // querying locally, so 2 queries to local nodes is likely to still be
@@ -71,10 +70,10 @@ public abstract class AbstractEndpointSnitch implements IEndpointSnitch
              : true;
     }
 
-    private boolean hasRemoteNode(List<InetAddress> l)
+    private boolean hasRemoteNode(List<InetAddressAndPort> l)
     {
         String localDc = DatabaseDescriptor.getLocalDataCenter();
-        for (InetAddress ep : l)
+        for (InetAddressAndPort ep : l)
         {
             if (!localDc.equals(getDatacenter(ep)))
                 return true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java b/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java
index b5606d6..e91f6ac 100644
--- a/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java
+++ b/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
-
 /**
  * An endpoint snitch tells Cassandra information about network topology that it can use to route
  * requests more efficiently.
@@ -30,16 +28,16 @@ public abstract class AbstractNetworkTopologySnitch extends AbstractEndpointSnit
      * @param endpoint a specified endpoint
      * @return string of rack
      */
-    abstract public String getRack(InetAddress endpoint);
+    abstract public String getRack(InetAddressAndPort endpoint);
 
     /**
      * Return the data center for which an endpoint resides in
      * @param endpoint a specified endpoint
      * @return string of data center
      */
-    abstract public String getDatacenter(InetAddress endpoint);
+    abstract public String getDatacenter(InetAddressAndPort endpoint);
 
-    public int compareEndpoints(InetAddress address, InetAddress a1, InetAddress a2)
+    public int compareEndpoints(InetAddressAndPort address, InetAddressAndPort a1, InetAddressAndPort a2)
     {
         if (address.equals(a1) && !address.equals(a2))
             return -1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index c3498d9..3e9b5bb 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.locator;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
-import java.net.InetAddress;
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -74,9 +73,9 @@ public abstract class AbstractReplicationStrategy
         // lazy-initialize keyspace itself since we don't create them until after the replication strategies
     }
 
-    private final Map<Token, ArrayList<InetAddress>> cachedEndpoints = new NonBlockingHashMap<Token, ArrayList<InetAddress>>();
+    private final Map<Token, ArrayList<InetAddressAndPort>> cachedEndpoints = new NonBlockingHashMap<Token, ArrayList<InetAddressAndPort>>();
 
-    public ArrayList<InetAddress> getCachedEndpoints(Token t)
+    public ArrayList<InetAddressAndPort> getCachedEndpoints(Token t)
     {
         long lastVersion = tokenMetadata.getRingVersion();
 
@@ -103,21 +102,21 @@ public abstract class AbstractReplicationStrategy
      * @param searchPosition the position the natural endpoints are requested for
      * @return a copy of the natural endpoints for the given token
      */
-    public ArrayList<InetAddress> getNaturalEndpoints(RingPosition searchPosition)
+    public ArrayList<InetAddressAndPort> getNaturalEndpoints(RingPosition searchPosition)
     {
         Token searchToken = searchPosition.getToken();
         Token keyToken = TokenMetadata.firstToken(tokenMetadata.sortedTokens(), searchToken);
-        ArrayList<InetAddress> endpoints = getCachedEndpoints(keyToken);
+        ArrayList<InetAddressAndPort> endpoints = getCachedEndpoints(keyToken);
         if (endpoints == null)
         {
             TokenMetadata tm = tokenMetadata.cachedOnlyTokenMap();
             // if our cache got invalidated, it's possible there is a new token to account for too
             keyToken = TokenMetadata.firstToken(tm.sortedTokens(), searchToken);
-            endpoints = new ArrayList<InetAddress>(calculateNaturalEndpoints(searchToken, tm));
+            endpoints = new ArrayList<InetAddressAndPort>(calculateNaturalEndpoints(searchToken, tm));
             cachedEndpoints.put(keyToken, endpoints);
         }
 
-        return new ArrayList<InetAddress>(endpoints);
+        return new ArrayList<InetAddressAndPort>(endpoints);
     }
 
     /**
@@ -128,10 +127,10 @@ public abstract class AbstractReplicationStrategy
      * @param searchToken the token the natural endpoints are requested for
      * @return a copy of the natural endpoints for the given token
      */
-    public abstract List<InetAddress> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata);
+    public abstract List<InetAddressAndPort> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata);
 
-    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
-                                                                       Collection<InetAddress> pendingEndpoints,
+    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(Collection<InetAddressAndPort> naturalEndpoints,
+                                                                       Collection<InetAddressAndPort> pendingEndpoints,
                                                                        ConsistencyLevel consistency_level,
                                                                        Runnable callback,
                                                                        WriteType writeType,
@@ -140,8 +139,8 @@ public abstract class AbstractReplicationStrategy
         return getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, callback, writeType, queryStartNanoTime, DatabaseDescriptor.getIdealConsistencyLevel());
     }
 
-    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
-                                                                       Collection<InetAddress> pendingEndpoints,
+    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(Collection<InetAddressAndPort> naturalEndpoints,
+                                                                       Collection<InetAddressAndPort> pendingEndpoints,
                                                                        ConsistencyLevel consistency_level,
                                                                        Runnable callback,
                                                                        WriteType writeType,
@@ -211,14 +210,14 @@ public abstract class AbstractReplicationStrategy
      * (fixing this would probably require merging tokenmetadata into replicationstrategy,
      * so we could cache/invalidate cleanly.)
      */
-    public Multimap<InetAddress, Range<Token>> getAddressRanges(TokenMetadata metadata)
+    public Multimap<InetAddressAndPort, Range<Token>> getAddressRanges(TokenMetadata metadata)
     {
-        Multimap<InetAddress, Range<Token>> map = HashMultimap.create();
+        Multimap<InetAddressAndPort, Range<Token>> map = HashMultimap.create();
 
         for (Token token : metadata.sortedTokens())
         {
             Range<Token> range = metadata.getPrimaryRangeFor(token);
-            for (InetAddress ep : calculateNaturalEndpoints(token, metadata))
+            for (InetAddressAndPort ep : calculateNaturalEndpoints(token, metadata))
             {
                 map.put(ep, range);
             }
@@ -227,14 +226,14 @@ public abstract class AbstractReplicationStrategy
         return map;
     }
 
-    public Multimap<Range<Token>, InetAddress> getRangeAddresses(TokenMetadata metadata)
+    public Multimap<Range<Token>, InetAddressAndPort> getRangeAddresses(TokenMetadata metadata)
     {
-        Multimap<Range<Token>, InetAddress> map = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> map = HashMultimap.create();
 
         for (Token token : metadata.sortedTokens())
         {
             Range<Token> range = metadata.getPrimaryRangeFor(token);
-            for (InetAddress ep : calculateNaturalEndpoints(token, metadata))
+            for (InetAddressAndPort ep : calculateNaturalEndpoints(token, metadata))
             {
                 map.put(range, ep);
             }
@@ -243,17 +242,17 @@ public abstract class AbstractReplicationStrategy
         return map;
     }
 
-    public Multimap<InetAddress, Range<Token>> getAddressRanges()
+    public Multimap<InetAddressAndPort, Range<Token>> getAddressRanges()
     {
         return getAddressRanges(tokenMetadata.cloneOnlyTokenMap());
     }
 
-    public Collection<Range<Token>> getPendingAddressRanges(TokenMetadata metadata, Token pendingToken, InetAddress pendingAddress)
+    public Collection<Range<Token>> getPendingAddressRanges(TokenMetadata metadata, Token pendingToken, InetAddressAndPort pendingAddress)
     {
         return getPendingAddressRanges(metadata, Arrays.asList(pendingToken), pendingAddress);
     }
 
-    public Collection<Range<Token>> getPendingAddressRanges(TokenMetadata metadata, Collection<Token> pendingTokens, InetAddress pendingAddress)
+    public Collection<Range<Token>> getPendingAddressRanges(TokenMetadata metadata, Collection<Token> pendingTokens, InetAddressAndPort pendingAddress)
     {
         TokenMetadata temp = metadata.cloneOnlyTokenMap();
         temp.updateNormalTokens(pendingTokens, pendingAddress);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/CloudstackSnitch.java b/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
index ec2e87e..be6d3c4 100644
--- a/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
+++ b/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
@@ -24,7 +24,6 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.io.File;
 import java.net.HttpURLConnection;
-import java.net.InetAddress;
 import java.net.URL;
 import java.net.URI;
 import java.nio.charset.StandardCharsets;
@@ -56,7 +55,7 @@ public class CloudstackSnitch extends AbstractNetworkTopologySnitch
     protected static final Logger logger = LoggerFactory.getLogger(CloudstackSnitch.class);
     protected static final String ZONE_NAME_QUERY_URI = "/latest/meta-data/availability-zone";
 
-    private Map<InetAddress, Map<String, String>> savedEndpoints;
+    private Map<InetAddressAndPort, Map<String, String>> savedEndpoints;
 
     private static final String DEFAULT_DC = "UNKNOWN-DC";
     private static final String DEFAULT_RACK = "UNKNOWN-RACK";
@@ -83,9 +82,9 @@ public class CloudstackSnitch extends AbstractNetworkTopologySnitch
         csZoneRack = zone_parts[2];
     }
 
-    public String getRack(InetAddress endpoint)
+    public String getRack(InetAddressAndPort endpoint)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
             return csZoneRack;
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         if (state == null || state.getApplicationState(ApplicationState.RACK) == null)
@@ -99,9 +98,9 @@ public class CloudstackSnitch extends AbstractNetworkTopologySnitch
         return state.getApplicationState(ApplicationState.RACK).value;
     }
 
-    public String getDatacenter(InetAddress endpoint)
+    public String getDatacenter(InetAddressAndPort endpoint)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
             return csZoneDc;
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         if (state == null || state.getApplicationState(ApplicationState.DC) == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
index 42fc26c..b9c9ba0 100644
--- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
@@ -25,6 +25,7 @@ import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import com.codahale.metrics.ExponentiallyDecayingReservoir;
 import javax.management.MBeanServer;
@@ -63,8 +64,8 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
     private String mbeanName;
     private boolean registered = false;
 
-    private volatile HashMap<InetAddress, Double> scores = new HashMap<>();
-    private final ConcurrentHashMap<InetAddress, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
+    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
+    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
 
     public final IEndpointSnitch subsnitch;
 
@@ -174,27 +175,27 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         subsnitch.gossiperStarting();
     }
 
-    public String getRack(InetAddress endpoint)
+    public String getRack(InetAddressAndPort endpoint)
     {
         return subsnitch.getRack(endpoint);
     }
 
-    public String getDatacenter(InetAddress endpoint)
+    public String getDatacenter(InetAddressAndPort endpoint)
     {
         return subsnitch.getDatacenter(endpoint);
     }
 
-    public List<InetAddress> getSortedListByProximity(final InetAddress address, Collection<InetAddress> addresses)
+    public List<InetAddressAndPort> getSortedListByProximity(final InetAddressAndPort address, Collection<InetAddressAndPort> addresses)
     {
-        List<InetAddress> list = new ArrayList<InetAddress>(addresses);
+        List<InetAddressAndPort> list = new ArrayList<>(addresses);
         sortByProximity(address, list);
         return list;
     }
 
     @Override
-    public void sortByProximity(final InetAddress address, List<InetAddress> addresses)
+    public void sortByProximity(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
     {
-        assert address.equals(FBUtilities.getBroadcastAddress()); // we only know about ourself
+        assert address.equals(FBUtilities.getBroadcastAddressAndPort()); // we only know about ourself
         if (dynamicBadnessThreshold == 0)
         {
             sortByProximityWithScore(address, addresses);
@@ -205,32 +206,32 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         }
     }
 
-    private void sortByProximityWithScore(final InetAddress address, List<InetAddress> addresses)
+    private void sortByProximityWithScore(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
     {
         // Scores can change concurrently from a call to this method. But Collections.sort() expects
         // its comparator to be "stable", that is 2 endpoint should compare the same way for the duration
         // of the sort() call. As we copy the scores map on write, it is thus enough to alias the current
         // version of it during this call.
-        final HashMap<InetAddress, Double> scores = this.scores;
-        Collections.sort(addresses, new Comparator<InetAddress>()
+        final HashMap<InetAddressAndPort, Double> scores = this.scores;
+        Collections.sort(addresses, new Comparator<InetAddressAndPort>()
         {
-            public int compare(InetAddress a1, InetAddress a2)
+            public int compare(InetAddressAndPort a1, InetAddressAndPort a2)
             {
                 return compareEndpoints(address, a1, a2, scores);
             }
         });
     }
 
-    private void sortByProximityWithBadness(final InetAddress address, List<InetAddress> addresses)
+    private void sortByProximityWithBadness(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
     {
         if (addresses.size() < 2)
             return;
 
         subsnitch.sortByProximity(address, addresses);
-        HashMap<InetAddress, Double> scores = this.scores; // Make sure the score don't change in the middle of the loop below
+        HashMap<InetAddressAndPort, Double> scores = this.scores; // Make sure the score don't change in the middle of the loop below
                                                            // (which wouldn't really matter here but its cleaner that way).
         ArrayList<Double> subsnitchOrderedScores = new ArrayList<>(addresses.size());
-        for (InetAddress inet : addresses)
+        for (InetAddressAndPort inet : addresses)
         {
             Double score = scores.get(inet);
             if (score == null)
@@ -256,7 +257,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
     }
 
     // Compare endpoints given an immutable snapshot of the scores
-    private int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2, Map<InetAddress, Double> scores)
+    private int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2, Map<InetAddressAndPort, Double> scores)
     {
         Double scored1 = scores.get(a1);
         Double scored2 = scores.get(a2);
@@ -279,7 +280,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
             return 1;
     }
 
-    public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+    public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
     {
         // That function is fundamentally unsafe because the scores can change at any time and so the result of that
         // method is not stable for identical arguments. This is why we don't rely on super.sortByProximity() in
@@ -287,7 +288,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         throw new UnsupportedOperationException("You shouldn't wrap the DynamicEndpointSnitch (within itself or otherwise)");
     }
 
-    public void receiveTiming(InetAddress host, long latency) // this is cheap
+    public void receiveTiming(InetAddressAndPort host, long latency) // this is cheap
     {
         ExponentiallyDecayingReservoir sample = samples.get(host);
         if (sample == null)
@@ -315,23 +316,23 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         }
         double maxLatency = 1;
 
-        Map<InetAddress, Snapshot> snapshots = new HashMap<>(samples.size());
-        for (Map.Entry<InetAddress, ExponentiallyDecayingReservoir> entry : samples.entrySet())
+        Map<InetAddressAndPort, Snapshot> snapshots = new HashMap<>(samples.size());
+        for (Map.Entry<InetAddressAndPort, ExponentiallyDecayingReservoir> entry : samples.entrySet())
         {
             snapshots.put(entry.getKey(), entry.getValue().getSnapshot());
         }
 
         // We're going to weight the latency for each host against the worst one we see, to
         // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
-        HashMap<InetAddress, Double> newScores = new HashMap<>();
-        for (Map.Entry<InetAddress, Snapshot> entry : snapshots.entrySet())
+        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>();
+        for (Map.Entry<InetAddressAndPort, Snapshot> entry : snapshots.entrySet())
         {
             double mean = entry.getValue().getMedian();
             if (mean > maxLatency)
                 maxLatency = mean;
         }
         // now make another pass to do the weighting based on the maximums we found before
-        for (Map.Entry<InetAddress, Snapshot> entry : snapshots.entrySet())
+        for (Map.Entry<InetAddressAndPort, Snapshot> entry : snapshots.entrySet())
         {
             double score = entry.getValue().getMedian() / maxLatency;
             // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
@@ -351,6 +352,11 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
 
     public Map<InetAddress, Double> getScores()
     {
+        return scores.entrySet().stream().collect(Collectors.toMap(address -> address.getKey().address, Map.Entry::getValue));
+    }
+
+    public Map<InetAddressAndPort, Double> getScoresWithPort()
+    {
         return scores;
     }
 
@@ -374,7 +380,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
 
     public List<Double> dumpTimings(String hostname) throws UnknownHostException
     {
-        InetAddress host = InetAddress.getByName(hostname);
+        InetAddressAndPort host = InetAddressAndPort.getByName(hostname);
         ArrayList<Double> timings = new ArrayList<Double>();
         ExponentiallyDecayingReservoir sample = samples.get(host);
         if (sample != null)
@@ -390,7 +396,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         Gossiper.instance.addLocalApplicationState(ApplicationState.SEVERITY, StorageService.instance.valueFactory.severity(severity));
     }
 
-    private double getSeverity(InetAddress endpoint)
+    private double getSeverity(InetAddressAndPort endpoint)
     {
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         if (state == null)
@@ -405,10 +411,10 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
 
     public double getSeverity()
     {
-        return getSeverity(FBUtilities.getBroadcastAddress());
+        return getSeverity(FBUtilities.getBroadcastAddressAndPort());
     }
 
-    public boolean isWorthMergingForRangeQuery(List<InetAddress> merged, List<InetAddress> l1, List<InetAddress> l2)
+    public boolean isWorthMergingForRangeQuery(List<InetAddressAndPort> merged, List<InetAddressAndPort> l1, List<InetAddressAndPort> l2)
     {
         if (!subsnitch.isWorthMergingForRangeQuery(merged, l1, l2))
             return false;
@@ -428,10 +434,10 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
     }
 
     // Return the max score for the endpoint in the provided list, or -1.0 if no node have a score.
-    private double maxScore(List<InetAddress> endpoints)
+    private double maxScore(List<InetAddressAndPort> endpoints)
     {
         double maxScore = -1.0;
-        for (InetAddress endpoint : endpoints)
+        for (InetAddressAndPort endpoint : endpoints)
         {
             Double score = scores.get(endpoint);
             if (score == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java
index bfafa75..61f0d97 100644
--- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java
+++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java
@@ -24,6 +24,8 @@ import java.util.List;
 
 public interface DynamicEndpointSnitchMBean 
 {
+    public Map<InetAddressAndPort, Double> getScoresWithPort();
+    @Deprecated
     public Map<InetAddress, Double> getScores();
     public int getUpdateInterval();
     public int getResetInterval();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
index b32ca84..2a6c7e9 100644
--- a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.locator;
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.UnknownHostException;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -62,6 +63,16 @@ public class Ec2MultiRegionSnitch extends Ec2Snitch
     public void gossiperStarting()
     {
         super.gossiperStarting();
+        InetAddressAndPort address;
+        try
+        {
+            address = InetAddressAndPort.getByName(localPrivateAddress);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+        Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT, StorageService.instance.valueFactory.internalAddressAndPort(address));
         Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP, StorageService.instance.valueFactory.internalIP(localPrivateAddress));
         Gossiper.instance.register(new ReconnectableSnitchHelper(this, ec2region, true));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/Ec2Snitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Ec2Snitch.java b/src/java/org/apache/cassandra/locator/Ec2Snitch.java
index 59eb27b..c7324c8 100644
--- a/src/java/org/apache/cassandra/locator/Ec2Snitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2Snitch.java
@@ -21,7 +21,6 @@ import java.io.DataInputStream;
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.net.HttpURLConnection;
-import java.net.InetAddress;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.util.Map;
@@ -46,7 +45,7 @@ public class Ec2Snitch extends AbstractNetworkTopologySnitch
     protected static final String ZONE_NAME_QUERY_URL = "http://169.254.169.254/latest/meta-data/placement/availability-zone";
     private static final String DEFAULT_DC = "UNKNOWN-DC";
     private static final String DEFAULT_RACK = "UNKNOWN-RACK";
-    private Map<InetAddress, Map<String, String>> savedEndpoints;
+    private Map<InetAddressAndPort, Map<String, String>> savedEndpoints;
     protected String ec2zone;
     protected String ec2region;
 
@@ -92,9 +91,9 @@ public class Ec2Snitch extends AbstractNetworkTopologySnitch
         }
     }
 
-    public String getRack(InetAddress endpoint)
+    public String getRack(InetAddressAndPort endpoint)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
             return ec2zone;
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         if (state == null || state.getApplicationState(ApplicationState.RACK) == null)
@@ -108,9 +107,9 @@ public class Ec2Snitch extends AbstractNetworkTopologySnitch
         return state.getApplicationState(ApplicationState.RACK).value;
     }
 
-    public String getDatacenter(InetAddress endpoint)
+    public String getDatacenter(InetAddressAndPort endpoint)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
             return ec2region;
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         if (state == null || state.getApplicationState(ApplicationState.DC) == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java b/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java
index bbfabb6..c06d765 100644
--- a/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java
+++ b/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.locator;
 
 
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
@@ -44,22 +43,22 @@ public class EndpointSnitchInfo implements EndpointSnitchInfoMBean
 
     public String getDatacenter(String host) throws UnknownHostException
     {
-        return DatabaseDescriptor.getEndpointSnitch().getDatacenter(InetAddress.getByName(host));
+        return DatabaseDescriptor.getEndpointSnitch().getDatacenter(InetAddressAndPort.getByName(host));
     }
 
     public String getRack(String host) throws UnknownHostException
     {
-        return DatabaseDescriptor.getEndpointSnitch().getRack(InetAddress.getByName(host));
+        return DatabaseDescriptor.getEndpointSnitch().getRack(InetAddressAndPort.getByName(host));
     }
 
     public String getDatacenter()
     {
-        return DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        return DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
     }
 
     public String getRack()
     {
-        return DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress());
+        return DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddressAndPort());
     }
 
     public String getSnitchName()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java b/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java
index b4d3b19..1e1c500 100644
--- a/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java
+++ b/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java
@@ -21,7 +21,6 @@ import java.io.DataInputStream;
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.net.HttpURLConnection;
-import java.net.InetAddress;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.util.Map;
@@ -46,7 +45,7 @@ public class GoogleCloudSnitch extends AbstractNetworkTopologySnitch
     protected static final String ZONE_NAME_QUERY_URL = "http://metadata.google.internal/computeMetadata/v1/instance/zone";
     private static final String DEFAULT_DC = "UNKNOWN-DC";
     private static final String DEFAULT_RACK = "UNKNOWN-RACK";
-    private Map<InetAddress, Map<String, String>> savedEndpoints;
+    private Map<InetAddressAndPort, Map<String, String>> savedEndpoints;
     protected String gceZone;
     protected String gceRegion;
 
@@ -94,9 +93,9 @@ public class GoogleCloudSnitch extends AbstractNetworkTopologySnitch
         }
     }
 
-    public String getRack(InetAddress endpoint)
+    public String getRack(InetAddressAndPort endpoint)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
             return gceZone;
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         if (state == null || state.getApplicationState(ApplicationState.RACK) == null)
@@ -110,9 +109,9 @@ public class GoogleCloudSnitch extends AbstractNetworkTopologySnitch
         return state.getApplicationState(ApplicationState.RACK).value;
     }
 
-    public String getDatacenter(InetAddress endpoint)
+    public String getDatacenter(InetAddressAndPort endpoint)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
             return gceRegion;
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         if (state == null || state.getApplicationState(ApplicationState.DC) == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
index e2449ae..75b5685 100644
--- a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.Map;
 
@@ -45,7 +44,7 @@ public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch//
     private final boolean preferLocal;
     private final AtomicReference<ReconnectableSnitchHelper> snitchHelperReference;
 
-    private Map<InetAddress, Map<String, String>> savedEndpoints;
+    private Map<InetAddressAndPort, Map<String, String>> savedEndpoints;
     private static final String DEFAULT_DC = "UNKNOWN_DC";
     private static final String DEFAULT_RACK = "UNKNOWN_RACK";
 
@@ -84,9 +83,9 @@ public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch//
      * @param endpoint the endpoint to process
      * @return string of data center
      */
-    public String getDatacenter(InetAddress endpoint)
+    public String getDatacenter(InetAddressAndPort endpoint)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
             return myDC;
 
         EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
@@ -112,9 +111,9 @@ public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch//
      * @param endpoint the endpoint to process
      * @return string of rack
      */
-    public String getRack(InetAddress endpoint)
+    public String getRack(InetAddressAndPort endpoint)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
             return myRack;
 
         EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
@@ -138,8 +137,10 @@ public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch//
     {
         super.gossiperStarting();
 
+        Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT,
+                                                   StorageService.instance.valueFactory.internalAddressAndPort(FBUtilities.getLocalAddressAndPort()));
         Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP,
-                StorageService.instance.valueFactory.internalIP(FBUtilities.getLocalAddress().getHostAddress()));
+                StorageService.instance.valueFactory.internalIP(FBUtilities.getJustLocalAddress().getHostAddress()));
 
         loadGossiperState();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/IEndpointSnitch.java b/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
index 71b441c..00a1543 100644
--- a/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.List;
 
@@ -32,27 +31,27 @@ public interface IEndpointSnitch
     /**
      * returns a String representing the rack this endpoint belongs to
      */
-    public String getRack(InetAddress endpoint);
+    public String getRack(InetAddressAndPort endpoint);
 
     /**
      * returns a String representing the datacenter this endpoint belongs to
      */
-    public String getDatacenter(InetAddress endpoint);
+    public String getDatacenter(InetAddressAndPort endpoint);
 
     /**
      * returns a new <tt>List</tt> sorted by proximity to the given endpoint
      */
-    public List<InetAddress> getSortedListByProximity(InetAddress address, Collection<InetAddress> unsortedAddress);
+    public List<InetAddressAndPort> getSortedListByProximity(InetAddressAndPort address, Collection<InetAddressAndPort> unsortedAddress);
 
     /**
      * This method will sort the <tt>List</tt> by proximity to the given address.
      */
-    public void sortByProximity(InetAddress address, List<InetAddress> addresses);
+    public void sortByProximity(InetAddressAndPort address, List<InetAddressAndPort> addresses);
 
     /**
      * compares two endpoints in relation to the target endpoint, returning as Comparator.compare would
      */
-    public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2);
+    public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2);
 
     /**
      * called after Gossiper instance exists immediately before it starts gossiping
@@ -63,5 +62,5 @@ public interface IEndpointSnitch
      * Returns whether for a range query doing a query against merged is likely
      * to be faster than 2 sequential queries, one against l1 followed by one against l2.
      */
-    public boolean isWorthMergingForRangeQuery(List<InetAddress> merged, List<InetAddress> l1, List<InetAddress> l2);
+    public boolean isWorthMergingForRangeQuery(List<InetAddressAndPort> merged, List<InetAddressAndPort> l1, List<InetAddressAndPort> l2);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/ILatencySubscriber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/ILatencySubscriber.java b/src/java/org/apache/cassandra/locator/ILatencySubscriber.java
index d2ae6db..f6c1c7f 100644
--- a/src/java/org/apache/cassandra/locator/ILatencySubscriber.java
+++ b/src/java/org/apache/cassandra/locator/ILatencySubscriber.java
@@ -17,9 +17,7 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
-
 public interface ILatencySubscriber
 {
-    public void receiveTiming(InetAddress address, long latency);
+    public void receiveTiming(InetAddressAndPort address, long latency);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
new file mode 100644
index 0000000..38a1a49
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
@@ -0,0 +1,203 @@
+/*
+ * 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.locator;
+
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
+
+import org.apache.cassandra.utils.FastByteOperations;
+
+/**
+ * A class to replace the usage of InetAddress to identify hosts in the cluster.
+ * Opting for a full replacement class so that in the future if we change the nature
+ * of the identifier the refactor will be easier in that we don't have to change the type
+ * just the methods.
+ *
+ * Because an IP might contain multiple C* instances the identification must be done
+ * using the IP + port. InetSocketAddress is undesirable for a couple of reasons. It's not comparable,
+ * it's toString() method doesn't correctly bracket IPv6, it doesn't handle optional default values,
+ * and a couple of other minor behaviors that are slightly less troublesome like handling the
+ * need to sometimes return a port and sometimes not.
+ *
+ */
+public final class InetAddressAndPort implements Comparable<InetAddressAndPort>, Serializable
+{
+    private static final long serialVersionUID = 0;
+
+    //Store these here to avoid requiring DatabaseDescriptor to be loaded. DatabaseDescriptor will set
+    //these when it loads the config. A lot of unit tests won't end up loading DatabaseDescriptor.
+    //Tools that might use this class also might not load database descriptor. Those tools are expected
+    //to always override the defaults.
+    static volatile int defaultPort = 7000;
+
+    public final InetAddress address;
+    public final byte[] addressBytes;
+    public final int port;
+
+    private InetAddressAndPort(InetAddress address, byte[] addressBytes, int port)
+    {
+        Preconditions.checkNotNull(address);
+        Preconditions.checkNotNull(addressBytes);
+        validatePortRange(port);
+        this.address = address;
+        this.port = port;
+        this.addressBytes = addressBytes;
+    }
+
+    private static void validatePortRange(int port)
+    {
+        if (port < 0 | port > 65535)
+        {
+            throw new IllegalArgumentException("Port " + port + " is not a valid port number in the range 0-65535");
+        }
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        InetAddressAndPort that = (InetAddressAndPort) o;
+
+        if (port != that.port) return false;
+        return address.equals(that.address);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = address.hashCode();
+        result = 31 * result + port;
+        return result;
+    }
+
+    @Override
+    public int compareTo(InetAddressAndPort o)
+    {
+        int retval = FastByteOperations.compareUnsigned(addressBytes, 0, addressBytes.length, o.addressBytes, 0, o.addressBytes.length);
+        if (retval != 0)
+        {
+            return retval;
+        }
+
+        return Integer.compare(port, o.port);
+    }
+
+    public String getHostAddress(boolean withPort)
+    {
+        if (withPort)
+        {
+            return toString();
+        }
+        else
+        {
+            return address.getHostAddress();
+        }
+    }
+
+    @Override
+    public String toString()
+    {
+        return toString(true);
+    }
+
+    public String toString(boolean withPort)
+    {
+        if (withPort)
+        {
+            return HostAndPort.fromParts(address.getHostAddress(), port).toString();
+        }
+        else
+        {
+            return address.toString();
+        }
+    }
+
+    public static InetAddressAndPort getByName(String name) throws UnknownHostException
+    {
+        return getByNameOverrideDefaults(name, null);
+    }
+
+    /**
+     *
+     * @param name Hostname + optional ports string
+     * @param port Port to connect on, overridden by values in hostname string, defaults to DatabaseDescriptor default if not specified anywhere.
+     * @return
+     * @throws UnknownHostException
+     */
+    public static InetAddressAndPort getByNameOverrideDefaults(String name, Integer port) throws UnknownHostException
+    {
+        HostAndPort hap = HostAndPort.fromString(name);
+        if (hap.hasPort())
+        {
+            port = hap.getPort();
+        }
+        return getByAddressOverrideDefaults(InetAddress.getByName(hap.getHost()), port);
+    }
+
+    public static InetAddressAndPort getByAddress(byte[] address) throws UnknownHostException
+    {
+        return getByAddressOverrideDefaults(InetAddress.getByAddress(address), address, null);
+    }
+
+    public static InetAddressAndPort getByAddress(InetAddress address)
+    {
+        return getByAddressOverrideDefaults(address, null);
+    }
+
+    public static InetAddressAndPort getByAddressOverrideDefaults(InetAddress address, Integer port)
+    {
+        if (port == null)
+        {
+            port = defaultPort;
+        }
+
+        return new InetAddressAndPort(address, address.getAddress(), port);
+    }
+
+    public static InetAddressAndPort getByAddressOverrideDefaults(InetAddress address, byte[] addressBytes, Integer port)
+    {
+        if (port == null)
+        {
+            port = defaultPort;
+        }
+
+        return new InetAddressAndPort(address, addressBytes, port);
+    }
+
+    public static InetAddressAndPort getLoopbackAddress()
+    {
+        return InetAddressAndPort.getByAddress(InetAddress.getLoopbackAddress());
+    }
+
+    public static InetAddressAndPort getLocalHost() throws UnknownHostException
+    {
+        return InetAddressAndPort.getByAddress(InetAddress.getLocalHost());
+    }
+
+    public static void initializeDefaultPort(int port)
+    {
+        defaultPort = port;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/LocalStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/LocalStrategy.java b/src/java/org/apache/cassandra/locator/LocalStrategy.java
index ae58203..a76fe96 100644
--- a/src/java/org/apache/cassandra/locator/LocalStrategy.java
+++ b/src/java/org/apache/cassandra/locator/LocalStrategy.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collection;
@@ -42,16 +41,16 @@ public class LocalStrategy extends AbstractReplicationStrategy
      * LocalStrategy may be used before tokens are set up.
      */
     @Override
-    public ArrayList<InetAddress> getNaturalEndpoints(RingPosition searchPosition)
+    public ArrayList<InetAddressAndPort> getNaturalEndpoints(RingPosition searchPosition)
     {
-        ArrayList<InetAddress> l = new ArrayList<InetAddress>(1);
-        l.add(FBUtilities.getBroadcastAddress());
+        ArrayList<InetAddressAndPort> l = new ArrayList<InetAddressAndPort>(1);
+        l.add(FBUtilities.getBroadcastAddressAndPort());
         return l;
     }
 
-    public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+    public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
     {
-        return Collections.singletonList(FBUtilities.getBroadcastAddress());
+        return Collections.singletonList(FBUtilities.getBroadcastAddressAndPort());
     }
 
     public int getReplicationFactor()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 442e6cf..673c018 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.*;
 import java.util.Map.Entry;
 
@@ -72,7 +71,7 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         }
 
         datacenters = Collections.unmodifiableMap(newDatacenters);
-        logger.trace("Configured datacenter replicas are {}", FBUtilities.toString(datacenters));
+        logger.info("Configured datacenter replicas are {}", FBUtilities.toString(datacenters));
     }
 
     /**
@@ -81,7 +80,7 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
     private static final class DatacenterEndpoints
     {
         /** List accepted endpoints get pushed into. */
-        Set<InetAddress> endpoints;
+        Set<InetAddressAndPort> endpoints;
         /**
          * Racks encountered so far. Replicas are put into separate racks while possible.
          * For efficiency the set is shared between the instances, using the location pair (dc, rack) to make sure
@@ -93,7 +92,7 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         int rfLeft;
         int acceptableRackRepeats;
 
-        DatacenterEndpoints(int rf, int rackCount, int nodeCount, Set<InetAddress> endpoints, Set<Pair<String, String>> racks)
+        DatacenterEndpoints(int rf, int rackCount, int nodeCount, Set<InetAddressAndPort> endpoints, Set<Pair<String, String>> racks)
         {
             this.endpoints = endpoints;
             this.racks = racks;
@@ -108,7 +107,7 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
          * Attempts to add an endpoint to the replicas for this datacenter, adding to the endpoints set if successful.
          * Returns true if the endpoint was added, and this datacenter does not require further replicas.
          */
-        boolean addEndpointAndCheckIfDone(InetAddress ep, Pair<String,String> location)
+        boolean addEndpointAndCheckIfDone(InetAddressAndPort ep, Pair<String,String> location)
         {
             if (done())
                 return false;
@@ -143,17 +142,17 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
     /**
      * calculate endpoints in one pass through the tokens by tracking our progress in each DC.
      */
-    public List<InetAddress> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata)
+    public List<InetAddressAndPort> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata)
     {
         // we want to preserve insertion order so that the first added endpoint becomes primary
-        Set<InetAddress> replicas = new LinkedHashSet<>();
+        Set<InetAddressAndPort> replicas = new LinkedHashSet<>();
         Set<Pair<String, String>> seenRacks = new HashSet<>();
 
         Topology topology = tokenMetadata.getTopology();
         // all endpoints in each DC, so we can check when we have exhausted all the members of a DC
-        Multimap<String, InetAddress> allEndpoints = topology.getDatacenterEndpoints();
+        Multimap<String, InetAddressAndPort> allEndpoints = topology.getDatacenterEndpoints();
         // all racks in a DC so we can check when we have exhausted all racks in a DC
-        Map<String, Multimap<String, InetAddress>> racks = topology.getDatacenterRacks();
+        Map<String, Multimap<String, InetAddressAndPort>> racks = topology.getDatacenterRacks();
         assert !allEndpoints.isEmpty() && !racks.isEmpty() : "not aware of any cluster members";
 
         int dcsToFill = 0;
@@ -178,7 +177,7 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         while (dcsToFill > 0 && tokenIter.hasNext())
         {
             Token next = tokenIter.next();
-            InetAddress ep = tokenMetadata.getEndpoint(next);
+            InetAddressAndPort ep = tokenMetadata.getEndpoint(next);
             Pair<String, String> location = topology.getLocation(ep);
             DatacenterEndpoints dcEndpoints = dcs.get(location.left);
             if (dcEndpoints != null && dcEndpoints.addEndpointAndCheckIfDone(ep, location))
@@ -227,9 +226,9 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
 
         // Add data center of localhost.
-        validDataCenters.add(snitch.getDatacenter(FBUtilities.getBroadcastAddress()));
+        validDataCenters.add(snitch.getDatacenter(FBUtilities.getBroadcastAddressAndPort()));
         // Fetch and add DCs of all peers.
-        for (final InetAddress peer : StorageService.instance.getTokenMetadata().getAllEndpoints())
+        for (final InetAddressAndPort peer : StorageService.instance.getTokenMetadata().getAllEndpoints())
         {
             validDataCenters.add(snitch.getDatacenter(peer));
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
index b9bd767..93e629e 100644
--- a/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collection;
@@ -42,10 +41,10 @@ public class OldNetworkTopologyStrategy extends AbstractReplicationStrategy
         super(keyspaceName, tokenMetadata, snitch, configOptions);
     }
 
-    public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+    public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
     {
         int replicas = getReplicationFactor();
-        List<InetAddress> endpoints = new ArrayList<InetAddress>(replicas);
+        List<InetAddressAndPort> endpoints = new ArrayList<>(replicas);
         ArrayList<Token> tokens = metadata.sortedTokens();
 
         if (tokens.isEmpty())


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


[09/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 89d5358..cda575a 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -158,23 +158,23 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public Collection<Range<Token>> getLocalRanges(String keyspaceName)
     {
-        return getRangesForEndpoint(keyspaceName, FBUtilities.getBroadcastAddress());
+        return getRangesForEndpoint(keyspaceName, FBUtilities.getBroadcastAddressAndPort());
     }
 
     public Collection<Range<Token>> getPrimaryRanges(String keyspace)
     {
-        return getPrimaryRangesForEndpoint(keyspace, FBUtilities.getBroadcastAddress());
+        return getPrimaryRangesForEndpoint(keyspace, FBUtilities.getBroadcastAddressAndPort());
     }
 
     public Collection<Range<Token>> getPrimaryRangesWithinDC(String keyspace)
     {
-        return getPrimaryRangeForEndpointWithinDC(keyspace, FBUtilities.getBroadcastAddress());
+        return getPrimaryRangeForEndpointWithinDC(keyspace, FBUtilities.getBroadcastAddressAndPort());
     }
 
-    private final Set<InetAddress> replicatingNodes = Collections.synchronizedSet(new HashSet<InetAddress>());
+    private final Set<InetAddressAndPort> replicatingNodes = Collections.synchronizedSet(new HashSet<InetAddressAndPort>());
     private CassandraDaemon daemon;
 
-    private InetAddress removingNode;
+    private InetAddressAndPort removingNode;
 
     /* Are we starting this node in bootstrap mode? */
     private volatile boolean isBootstrapMode;
@@ -225,7 +225,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         SystemKeyspace.updateTokens(tokens);
         Collection<Token> localTokens = getLocalTokens();
         setGossipTokens(localTokens);
-        tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddress());
+        tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddressAndPort());
         setMode(Mode.NORMAL, false);
     }
 
@@ -233,6 +233,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         List<Pair<ApplicationState, VersionedValue>> states = new ArrayList<Pair<ApplicationState, VersionedValue>>();
         states.add(Pair.create(ApplicationState.TOKENS, valueFactory.tokens(tokens)));
+        states.add(Pair.create(ApplicationState.STATUS_WITH_PORT, valueFactory.normal(tokens)));
         states.add(Pair.create(ApplicationState.STATUS, valueFactory.normal(tokens)));
         Gossiper.instance.addLocalApplicationStates(states);
     }
@@ -407,7 +408,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * they get the Gossip shutdown message, so even if
      * we don't get time to broadcast this, it is not a problem.
      *
-     * See {@link Gossiper#markAsShutdown(InetAddress)}
+     * See {@link Gossiper#markAsShutdown(InetAddressAndPort)}
      */
     private void shutdownClientServers()
     {
@@ -463,9 +464,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                        "To perform this operation, please restart with " +
                                        "-Dcassandra.allow_unsafe_replace=true");
 
-        InetAddress replaceAddress = DatabaseDescriptor.getReplaceAddress();
+        InetAddressAndPort replaceAddress = DatabaseDescriptor.getReplaceAddress();
         logger.info("Gathering node replacement information for {}", replaceAddress);
-        Map<InetAddress, EndpointState> epStates = Gossiper.instance.doShadowRound();
+        Map<InetAddressAndPort, EndpointState> epStates = Gossiper.instance.doShadowRound();
         // as we've completed the shadow round of gossip, we should be able to find the node we're replacing
         if (epStates.get(replaceAddress) == null)
             throw new RuntimeException(String.format("Cannot replace_address %s because it doesn't exist in gossip", replaceAddress));
@@ -503,25 +504,32 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
 
         logger.debug("Starting shadow gossip round to check for endpoint collision");
-        Map<InetAddress, EndpointState> epStates = Gossiper.instance.doShadowRound();
+        Map<InetAddressAndPort, EndpointState> epStates = Gossiper.instance.doShadowRound();
         // If bootstrapping, check whether any previously known status for the endpoint makes it unsafe to do so.
         // If not bootstrapping, compare the host id for this endpoint learned from gossip (if any) with the local
         // one, which was either read from system.local or generated at startup. If a learned id is present &
         // doesn't match the local, then the node needs replacing
-        if (!Gossiper.instance.isSafeForStartup(FBUtilities.getBroadcastAddress(), localHostId, shouldBootstrap(), epStates))
+        if (!Gossiper.instance.isSafeForStartup(FBUtilities.getBroadcastAddressAndPort(), localHostId, shouldBootstrap(), epStates))
         {
             throw new RuntimeException(String.format("A node with address %s already exists, cancelling join. " +
                                                      "Use cassandra.replace_address if you want to replace this node.",
-                                                     FBUtilities.getBroadcastAddress()));
+                                                     FBUtilities.getBroadcastAddressAndPort()));
         }
 
         if (shouldBootstrap() && useStrictConsistency && !allowSimultaneousMoves())
         {
-            for (Map.Entry<InetAddress, EndpointState> entry : epStates.entrySet())
+            for (Map.Entry<InetAddressAndPort, EndpointState> entry : epStates.entrySet())
             {
                 // ignore local node or empty status
-                if (entry.getKey().equals(FBUtilities.getBroadcastAddress()) || entry.getValue().getApplicationState(ApplicationState.STATUS) == null)
+                if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()) || (entry.getValue().getApplicationState(ApplicationState.STATUS_WITH_PORT) == null & entry.getValue().getApplicationState(ApplicationState.STATUS) == null))
                     continue;
+
+                VersionedValue value = entry.getValue().getApplicationState(ApplicationState.STATUS_WITH_PORT);
+                if (value == null)
+                {
+                    value = entry.getValue().getApplicationState(ApplicationState.STATUS);
+                }
+
                 String[] pieces = splitValue(entry.getValue().getApplicationState(ApplicationState.STATUS));
                 assert (pieces.length > 0);
                 String state = pieces[0];
@@ -553,10 +561,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (Boolean.parseBoolean(System.getProperty("cassandra.load_ring_state", "true")))
         {
             logger.info("Populating token metadata from system tables");
-            Multimap<InetAddress, Token> loadedTokens = SystemKeyspace.loadTokens();
+            Multimap<InetAddressAndPort, Token> loadedTokens = SystemKeyspace.loadTokens();
             if (!shouldBootstrap()) // if we have not completed bootstrapping, we should not add ourselves as a normal token
-                loadedTokens.putAll(FBUtilities.getBroadcastAddress(), SystemKeyspace.getSavedTokens());
-            for (InetAddress ep : loadedTokens.keySet())
+                loadedTokens.putAll(FBUtilities.getBroadcastAddressAndPort(), SystemKeyspace.getSavedTokens());
+            for (InetAddressAndPort ep : loadedTokens.keySet())
                 tokenMetadata.updateNormalTokens(loadedTokens.get(ep), ep);
 
             logger.info("Token metadata: {}", tokenMetadata);
@@ -640,10 +648,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             Collection<Token> tokens = SystemKeyspace.getSavedTokens();
             if (!tokens.isEmpty())
             {
-                tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddress());
+                tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddressAndPort());
                 // order is important here, the gossiper can fire in between adding these two states.  It's ok to send TOKENS without STATUS, but *not* vice versa.
                 List<Pair<ApplicationState, VersionedValue>> states = new ArrayList<Pair<ApplicationState, VersionedValue>>();
                 states.add(Pair.create(ApplicationState.TOKENS, valueFactory.tokens(tokens)));
+                states.add(Pair.create(ApplicationState.STATUS_WITH_PORT, valueFactory.hibernate(true)));
                 states.add(Pair.create(ApplicationState.STATUS, valueFactory.hibernate(true)));
                 Gossiper.instance.addLocalApplicationStates(states);
             }
@@ -659,11 +668,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (Boolean.parseBoolean(System.getProperty("cassandra.load_ring_state", "true")))
         {
             logger.info("Loading persisted ring state");
-            Multimap<InetAddress, Token> loadedTokens = SystemKeyspace.loadTokens();
-            Map<InetAddress, UUID> loadedHostIds = SystemKeyspace.loadHostIds();
-            for (InetAddress ep : loadedTokens.keySet())
+            Multimap<InetAddressAndPort, Token> loadedTokens = SystemKeyspace.loadTokens();
+            Map<InetAddressAndPort, UUID> loadedHostIds = SystemKeyspace.loadHostIds();
+            for (InetAddressAndPort ep : loadedTokens.keySet())
             {
-                if (ep.equals(FBUtilities.getBroadcastAddress()))
+                if (ep.equals(FBUtilities.getBroadcastAddressAndPort()))
                 {
                     // entry has been mistakenly added, delete it
                     SystemKeyspace.removeEndpoint(ep);
@@ -707,7 +716,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public static boolean isSeed()
     {
-        return DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress());
+        return DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddressAndPort());
     }
 
     private void prepareToJoin() throws ConfigurationException
@@ -753,6 +762,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                 "the node to be replaced ({}). If the previous node has been down for longer than max_hint_window_in_ms, " +
                                 "repair must be run after the replacement process in order to make this node consistent.",
                                 DatabaseDescriptor.getReplaceAddress());
+                    appStates.put(ApplicationState.STATUS_WITH_PORT, valueFactory.hibernate(true));
                     appStates.put(ApplicationState.STATUS, valueFactory.hibernate(true));
                 }
             }
@@ -765,10 +775,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             // for bootstrap to get the load info it needs.
             // (we won't be part of the storage ring though until we add a counterId to our state, below.)
             // Seed the host ID-to-endpoint map with our own ID.
-            getTokenMetadata().updateHostId(localHostId, FBUtilities.getBroadcastAddress());
+            getTokenMetadata().updateHostId(localHostId, FBUtilities.getBroadcastAddressAndPort());
             appStates.put(ApplicationState.NET_VERSION, valueFactory.networkVersion());
             appStates.put(ApplicationState.HOST_ID, valueFactory.hostId(localHostId));
-            appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(FBUtilities.getBroadcastRpcAddress()));
+            appStates.put(ApplicationState.NATIVE_ADDRESS_AND_PORT, valueFactory.nativeaddressAndPort(FBUtilities.getBroadcastNativeAddressAndPort()));
+            appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(FBUtilities.getJustBroadcastNativeAddress()));
             appStates.put(ApplicationState.RELEASE_VERSION, valueFactory.releaseVersion());
 
             // load the persisted ring state. This used to be done earlier in the init process,
@@ -826,16 +837,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         //
         // We attempted to replace this with a schema-presence check, but you need a meaningful sleep
         // to get schema info from gossip which defeats the purpose.  See CASSANDRA-4427 for the gory details.
-        Set<InetAddress> current = new HashSet<>();
+        Set<InetAddressAndPort> current = new HashSet<>();
         if (logger.isDebugEnabled())
         {
             logger.debug("Bootstrap variables: {} {} {} {}",
                          DatabaseDescriptor.isAutoBootstrap(),
                          SystemKeyspace.bootstrapInProgress(),
                          SystemKeyspace.bootstrapComplete(),
-                         DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()));
+                         DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddressAndPort()));
         }
-        if (DatabaseDescriptor.isAutoBootstrap() && !SystemKeyspace.bootstrapComplete() && DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()))
+        if (DatabaseDescriptor.isAutoBootstrap() && !SystemKeyspace.bootstrapComplete() && DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddressAndPort()))
         {
             logger.info("This node will not auto bootstrap because it is configured to be a seed node.");
         }
@@ -873,13 +884,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             // get bootstrap tokens
             if (!replacing)
             {
-                if (tokenMetadata.isMember(FBUtilities.getBroadcastAddress()))
+                if (tokenMetadata.isMember(FBUtilities.getBroadcastAddressAndPort()))
                 {
                     String s = "This node is already a member of the token ring; bootstrap aborted. (If replacing a dead node, remove the old one from the ring first.)";
                     throw new UnsupportedOperationException(s);
                 }
                 setMode(Mode.JOINING, "getting bootstrap token", true);
-                bootstrapTokens = BootStrapper.getBootstrapTokens(tokenMetadata, FBUtilities.getBroadcastAddress(), delay);
+                bootstrapTokens = BootStrapper.getBootstrapTokens(tokenMetadata, FBUtilities.getBroadcastAddressAndPort(), delay);
             }
             else
             {
@@ -899,7 +910,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     // check for operator errors...
                     for (Token token : bootstrapTokens)
                     {
-                        InetAddress existing = tokenMetadata.getEndpoint(token);
+                        InetAddressAndPort existing = tokenMetadata.getEndpoint(token);
                         if (existing != null)
                         {
                             long nanoDelay = delay * 1000000L;
@@ -935,7 +946,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             bootstrapTokens = SystemKeyspace.getSavedTokens();
             if (bootstrapTokens.isEmpty())
             {
-                bootstrapTokens = BootStrapper.getBootstrapTokens(tokenMetadata, FBUtilities.getBroadcastAddress(), delay);
+                bootstrapTokens = BootStrapper.getBootstrapTokens(tokenMetadata, FBUtilities.getBroadcastAddressAndPort(), delay);
             }
             else
             {
@@ -958,7 +969,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 // remove the existing info about the replaced node.
                 if (!current.isEmpty())
                 {
-                    for (InetAddress existing : current)
+                    for (InetAddressAndPort existing : current)
                         Gossiper.instance.replacedEndpoint(existing);
                 }
             }
@@ -975,15 +986,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public static boolean isReplacingSameAddress()
     {
-        InetAddress replaceAddress = DatabaseDescriptor.getReplaceAddress();
-        return replaceAddress != null && replaceAddress.equals(FBUtilities.getBroadcastAddress());
+        InetAddressAndPort replaceAddress = DatabaseDescriptor.getReplaceAddress();
+        return replaceAddress != null && replaceAddress.equals(FBUtilities.getBroadcastAddressAndPort());
     }
 
     public void gossipSnitchInfo()
     {
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        String dc = snitch.getDatacenter(FBUtilities.getBroadcastAddress());
-        String rack = snitch.getRack(FBUtilities.getBroadcastAddress());
+        String dc = snitch.getDatacenter(FBUtilities.getBroadcastAddressAndPort());
+        String rack = snitch.getRack(FBUtilities.getBroadcastAddressAndPort());
         Gossiper.instance.addLocalApplicationState(ApplicationState.DC, StorageService.instance.valueFactory.datacenter(dc));
         Gossiper.instance.addLocalApplicationState(ApplicationState.RACK, StorageService.instance.valueFactory.rack(rack));
     }
@@ -1111,7 +1122,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public boolean isJoined()
     {
-        return tokenMetadata.isMember(FBUtilities.getBroadcastAddress()) && !isSurveyMode;
+        return tokenMetadata.isMember(FBUtilities.getBroadcastAddressAndPort()) && !isSurveyMode;
     }
 
     public void rebuild(String sourceDc)
@@ -1141,7 +1152,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             RangeStreamer streamer = new RangeStreamer(tokenMetadata,
                                                        null,
-                                                       FBUtilities.getBroadcastAddress(),
+                                                       FBUtilities.getBroadcastAddressAndPort(),
                                                        StreamOperation.REBUILD,
                                                        useStrictConsistency && !replacing,
                                                        DatabaseDescriptor.getEndpointSnitch(),
@@ -1202,13 +1213,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 if (specificSources != null)
                 {
                     String[] stringHosts = specificSources.split(",");
-                    Set<InetAddress> sources = new HashSet<>(stringHosts.length);
+                    Set<InetAddressAndPort> sources = new HashSet<>(stringHosts.length);
                     for (String stringHost : stringHosts)
                     {
                         try
                         {
-                            InetAddress endpoint = InetAddress.getByName(stringHost);
-                            if (FBUtilities.getBroadcastAddress().equals(endpoint))
+                            InetAddressAndPort endpoint = InetAddressAndPort.getByName(stringHost);
+                            if (FBUtilities.getBroadcastAddressAndPort().equals(endpoint))
                             {
                                 throw new IllegalArgumentException("This host was specified as a source for rebuilding. Sources for a rebuild can only be other nodes in the cluster.");
                             }
@@ -1449,8 +1460,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             // if not an existing token then bootstrap
             List<Pair<ApplicationState, VersionedValue>> states = new ArrayList<>();
             states.add(Pair.create(ApplicationState.TOKENS, valueFactory.tokens(tokens)));
+            states.add(Pair.create(ApplicationState.STATUS_WITH_PORT, replacing?
+                                                            valueFactory.bootReplacingWithPort(DatabaseDescriptor.getReplaceAddress()) :
+                                                            valueFactory.bootstrapping(tokens)));
             states.add(Pair.create(ApplicationState.STATUS, replacing?
-                                                            valueFactory.bootReplacing(DatabaseDescriptor.getReplaceAddress()) :
+                                                            valueFactory.bootReplacing(DatabaseDescriptor.getReplaceAddress().address) :
                                                             valueFactory.bootstrapping(tokens)));
             Gossiper.instance.addLocalApplicationStates(states);
             setMode(Mode.JOINING, "sleeping " + RING_DELAY + " ms for pending range setup", true);
@@ -1459,7 +1473,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         else
         {
             // Dont set any state for the node which is bootstrapping the existing token...
-            tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddress());
+            tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddressAndPort());
             SystemKeyspace.removeEndpoint(DatabaseDescriptor.getReplaceAddress());
         }
         if (!Gossiper.instance.seenAnySeed())
@@ -1475,7 +1489,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         invalidateDiskBoundaries();
 
         setMode(Mode.JOINING, "Starting to bootstrap...", true);
-        BootStrapper bootstrapper = new BootStrapper(FBUtilities.getBroadcastAddress(), tokens, tokenMetadata);
+        BootStrapper bootstrapper = new BootStrapper(FBUtilities.getBroadcastAddressAndPort(), tokens, tokenMetadata);
         bootstrapper.addProgressListener(progressSupport);
         ListenableFuture<StreamState> bootstrapStream = bootstrapper.bootstrap(streamStateStore, useStrictConsistency && !replacing); // handles token update
         Futures.addCallback(bootstrapStream, new FutureCallback<StreamState>()
@@ -1547,7 +1561,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             // get bootstrap tokens saved in system keyspace
             final Collection<Token> tokens = SystemKeyspace.getSavedTokens();
             // already bootstrapped ranges are filtered during bootstrap
-            BootStrapper bootstrapper = new BootStrapper(FBUtilities.getBroadcastAddress(), tokens, tokenMetadata);
+            BootStrapper bootstrapper = new BootStrapper(FBUtilities.getBroadcastAddressAndPort(), tokens, tokenMetadata);
             bootstrapper.addProgressListener(progressSupport);
             ListenableFuture<StreamState> bootstrapStream = bootstrapper.bootstrap(streamStateStore, useStrictConsistency && !replacing); // handles token update
             Futures.addCallback(bootstrapStream, new FutureCallback<StreamState>()
@@ -1608,35 +1622,67 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return tokenMetadata;
     }
 
+    public Map<List<String>, List<String>> getRangeToEndpointMap(String keyspace)
+    {
+        return getRangeToEndpointMap(keyspace, false);
+    }
+
+    public Map<List<String>, List<String>> getRangeToEndpointWithPortMap(String keyspace)
+    {
+         return getRangeToEndpointMap(keyspace, true);
+    }
+
     /**
      * for a keyspace, return the ranges and corresponding listen addresses.
      * @param keyspace
      * @return the endpoint map
      */
-    public Map<List<String>, List<String>> getRangeToEndpointMap(String keyspace)
+    public Map<List<String>, List<String>> getRangeToEndpointMap(String keyspace, boolean withPort)
     {
         /* All the ranges for the tokens */
         Map<List<String>, List<String>> map = new HashMap<>();
-        for (Map.Entry<Range<Token>,List<InetAddress>> entry : getRangeToAddressMap(keyspace).entrySet())
+        for (Map.Entry<Range<Token>,List<InetAddressAndPort>> entry : getRangeToAddressMap(keyspace).entrySet())
         {
-            map.put(entry.getKey().asList(), stringify(entry.getValue()));
+            map.put(entry.getKey().asList(), stringify(entry.getValue(), withPort));
         }
         return map;
     }
 
     /**
-     * Return the rpc address associated with an endpoint as a string.
+     * Return the native address associated with an endpoint as a string.
      * @param endpoint The endpoint to get rpc address for
-     * @return the rpc address
+     * @return the native address
      */
-    public String getRpcaddress(InetAddress endpoint)
+    public String getNativeaddress(InetAddressAndPort endpoint, boolean withPort)
     {
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
-            return FBUtilities.getBroadcastRpcAddress().getHostAddress();
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
+            return FBUtilities.getBroadcastNativeAddressAndPort().toString(withPort);
+        else if (Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.NATIVE_ADDRESS_AND_PORT) != null)
+        {
+            try
+            {
+                InetAddressAndPort address = InetAddressAndPort.getByName(Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS).value);
+                return address.getHostAddress(withPort);
+            }
+            catch (UnknownHostException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
         else if (Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS) == null)
-            return endpoint.getHostAddress();
+            return endpoint.address.getHostAddress() + ":" + DatabaseDescriptor.getNativeTransportPort();
         else
-            return Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS).value;
+            return Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS).value + ":" + DatabaseDescriptor.getNativeTransportPort();
+    }
+
+    public Map<List<String>, List<String>> getRangeToRpcaddressMap(String keyspace)
+    {
+        return getRangeToNativeaddressMap(keyspace, false);
+    }
+
+    public Map<List<String>, List<String>> getRangeToNativeaddressWithPortMap(String keyspace)
+    {
+        return getRangeToNativeaddressMap(keyspace, true);
     }
 
     /**
@@ -1644,16 +1690,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param keyspace
      * @return the endpoint map
      */
-    public Map<List<String>, List<String>> getRangeToRpcaddressMap(String keyspace)
+    private Map<List<String>, List<String>> getRangeToNativeaddressMap(String keyspace, boolean withPort)
     {
         /* All the ranges for the tokens */
         Map<List<String>, List<String>> map = new HashMap<>();
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : getRangeToAddressMap(keyspace).entrySet())
+        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : getRangeToAddressMap(keyspace).entrySet())
         {
             List<String> rpcaddrs = new ArrayList<>(entry.getValue().size());
-            for (InetAddress endpoint: entry.getValue())
+            for (InetAddressAndPort endpoint: entry.getValue())
             {
-                rpcaddrs.add(getRpcaddress(endpoint));
+                rpcaddrs.add(getNativeaddress(endpoint, withPort));
             }
             map.put(entry.getKey().asList(), rpcaddrs);
         }
@@ -1662,40 +1708,50 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public Map<List<String>, List<String>> getPendingRangeToEndpointMap(String keyspace)
     {
+        return getPendingRangeToEndpointMap(keyspace, false);
+    }
+
+    public Map<List<String>, List<String>> getPendingRangeToEndpointWithPortMap(String keyspace)
+    {
+        return getPendingRangeToEndpointMap(keyspace, true);
+    }
+
+    private Map<List<String>, List<String>> getPendingRangeToEndpointMap(String keyspace, boolean withPort)
+    {
         // some people just want to get a visual representation of things. Allow null and set it to the first
         // non-system keyspace.
         if (keyspace == null)
             keyspace = Schema.instance.getNonLocalStrategyKeyspaces().get(0);
 
         Map<List<String>, List<String>> map = new HashMap<>();
-        for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : tokenMetadata.getPendingRangesMM(keyspace).asMap().entrySet())
+        for (Map.Entry<Range<Token>, Collection<InetAddressAndPort>> entry : tokenMetadata.getPendingRangesMM(keyspace).asMap().entrySet())
         {
-            List<InetAddress> l = new ArrayList<>(entry.getValue());
-            map.put(entry.getKey().asList(), stringify(l));
+            List<InetAddressAndPort> l = new ArrayList<>(entry.getValue());
+            map.put(entry.getKey().asList(), stringify(l, withPort));
         }
         return map;
     }
 
-    public Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace)
+    public Map<Range<Token>, List<InetAddressAndPort>> getRangeToAddressMap(String keyspace)
     {
         return getRangeToAddressMap(keyspace, tokenMetadata.sortedTokens());
     }
 
-    public Map<Range<Token>, List<InetAddress>> getRangeToAddressMapInLocalDC(String keyspace)
+    public Map<Range<Token>, List<InetAddressAndPort>> getRangeToAddressMapInLocalDC(String keyspace)
     {
-        Predicate<InetAddress> isLocalDC = new Predicate<InetAddress>()
+        Predicate<InetAddressAndPort> isLocalDC = new Predicate<InetAddressAndPort>()
         {
-            public boolean apply(InetAddress address)
+            public boolean apply(InetAddressAndPort address)
             {
                 return isLocalDC(address);
             }
         };
 
-        Map<Range<Token>, List<InetAddress>> origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
-        Map<Range<Token>, List<InetAddress>> filteredMap = Maps.newHashMap();
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : origMap.entrySet())
+        Map<Range<Token>, List<InetAddressAndPort>> origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
+        Map<Range<Token>, List<InetAddressAndPort>> filteredMap = Maps.newHashMap();
+        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : origMap.entrySet())
         {
-            List<InetAddress> endpointsInLocalDC = Lists.newArrayList(Collections2.filter(entry.getValue(), isLocalDC));
+            List<InetAddressAndPort> endpointsInLocalDC = Lists.newArrayList(Collections2.filter(entry.getValue(), isLocalDC));
             filteredMap.put(entry.getKey(), endpointsInLocalDC);
         }
 
@@ -1707,21 +1763,21 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         List<Token> filteredTokens = Lists.newArrayList();
         for (Token token : tokenMetadata.sortedTokens())
         {
-            InetAddress endpoint = tokenMetadata.getEndpoint(token);
+            InetAddressAndPort endpoint = tokenMetadata.getEndpoint(token);
             if (isLocalDC(endpoint))
                 filteredTokens.add(token);
         }
         return filteredTokens;
     }
 
-    private boolean isLocalDC(InetAddress targetHost)
+    private boolean isLocalDC(InetAddressAndPort targetHost)
     {
         String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(targetHost);
-        String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
         return remoteDC.equals(localDC);
     }
 
-    private Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
+    private Map<Range<Token>, List<InetAddressAndPort>> getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
     {
         // some people just want to get a visual representation of things. Allow null and set it to the first
         // non-system keyspace.
@@ -1733,6 +1789,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     }
 
 
+    public List<String> describeRingJMX(String keyspace) throws IOException
+    {
+        return describeRingJMX(keyspace, false);
+    }
+
+    public List<String> describeRingWithPortJMX(String keyspace) throws IOException
+    {
+        return describeRingJMX(keyspace,true);
+    }
+
     /**
      * The same as {@code describeRing(String)} but converts TokenRange to the String for JMX compatibility
      *
@@ -1740,12 +1806,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      *
      * @return a List of TokenRange(s) converted to String for the given keyspace
      */
-    public List<String> describeRingJMX(String keyspace) throws IOException
+    private List<String> describeRingJMX(String keyspace, boolean withPort) throws IOException
     {
         List<TokenRange> tokenRanges;
         try
         {
-            tokenRanges = describeRing(keyspace);
+            tokenRanges = describeRing(keyspace, false, withPort);
         }
         catch (InvalidRequestException e)
         {
@@ -1754,7 +1820,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         List<String> result = new ArrayList<>(tokenRanges.size());
 
         for (TokenRange tokenRange : tokenRanges)
-            result.add(tokenRange.toString());
+            result.add(tokenRange.toString(withPort));
 
         return result;
     }
@@ -1770,7 +1836,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public List<TokenRange> describeRing(String keyspace) throws InvalidRequestException
     {
-        return describeRing(keyspace, false);
+        return describeRing(keyspace, false, false);
     }
 
     /**
@@ -1778,10 +1844,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public List<TokenRange> describeLocalRing(String keyspace) throws InvalidRequestException
     {
-        return describeRing(keyspace, true);
+        return describeRing(keyspace, true, false);
     }
 
-    private List<TokenRange> describeRing(String keyspace, boolean includeOnlyLocalDC) throws InvalidRequestException
+    private List<TokenRange> describeRing(String keyspace, boolean includeOnlyLocalDC, boolean withPort) throws InvalidRequestException
     {
         if (!Schema.instance.getKeyspaces().contains(keyspace))
             throw new InvalidRequestException("No such keyspace: " + keyspace);
@@ -1792,39 +1858,49 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         List<TokenRange> ranges = new ArrayList<>();
         Token.TokenFactory tf = getTokenFactory();
 
-        Map<Range<Token>, List<InetAddress>> rangeToAddressMap =
+        Map<Range<Token>, List<InetAddressAndPort>> rangeToAddressMap =
                 includeOnlyLocalDC
                         ? getRangeToAddressMapInLocalDC(keyspace)
                         : getRangeToAddressMap(keyspace);
 
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : rangeToAddressMap.entrySet())
-            ranges.add(TokenRange.create(tf, entry.getKey(), entry.getValue()));
+        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : rangeToAddressMap.entrySet())
+            ranges.add(TokenRange.create(tf, entry.getKey(), entry.getValue(), withPort));
 
         return ranges;
     }
 
     public Map<String, String> getTokenToEndpointMap()
     {
-        Map<Token, InetAddress> mapInetAddress = tokenMetadata.getNormalAndBootstrappingTokenToEndpointMap();
+        return getTokenToEndpointMap(false);
+    }
+
+    public Map<String, String> getTokenToEndpointWithPortMap()
+    {
+        return getTokenToEndpointMap(true);
+    }
+
+    private Map<String, String> getTokenToEndpointMap(boolean withPort)
+    {
+        Map<Token, InetAddressAndPort> mapInetAddress = tokenMetadata.getNormalAndBootstrappingTokenToEndpointMap();
         // in order to preserve tokens in ascending order, we use LinkedHashMap here
         Map<String, String> mapString = new LinkedHashMap<>(mapInetAddress.size());
         List<Token> tokens = new ArrayList<>(mapInetAddress.keySet());
         Collections.sort(tokens);
         for (Token token : tokens)
         {
-            mapString.put(token.toString(), mapInetAddress.get(token).getHostAddress());
+            mapString.put(token.toString(), mapInetAddress.get(token).getHostAddress(withPort));
         }
         return mapString;
     }
 
     public String getLocalHostId()
     {
-        return getTokenMetadata().getHostId(FBUtilities.getBroadcastAddress()).toString();
+        return getTokenMetadata().getHostId(FBUtilities.getBroadcastAddressAndPort()).toString();
     }
 
     public UUID getLocalHostUUID()
     {
-        return getTokenMetadata().getHostId(FBUtilities.getBroadcastAddress());
+        return getTokenMetadata().getHostId(FBUtilities.getBroadcastAddressAndPort());
     }
 
     public Map<String, String> getHostIdMap()
@@ -1832,19 +1908,40 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return getEndpointToHostId();
     }
 
+
     public Map<String, String> getEndpointToHostId()
     {
+        return getEndpointToHostId(false);
+    }
+
+    public Map<String, String> getEndpointWithPortToHostId()
+    {
+        return getEndpointToHostId(true);
+    }
+
+    private  Map<String, String> getEndpointToHostId(boolean withPort)
+    {
         Map<String, String> mapOut = new HashMap<>();
-        for (Map.Entry<InetAddress, UUID> entry : getTokenMetadata().getEndpointToHostIdMapForReading().entrySet())
-            mapOut.put(entry.getKey().getHostAddress(), entry.getValue().toString());
+        for (Map.Entry<InetAddressAndPort, UUID> entry : getTokenMetadata().getEndpointToHostIdMapForReading().entrySet())
+            mapOut.put(entry.getKey().getHostAddress(withPort), entry.getValue().toString());
         return mapOut;
     }
 
     public Map<String, String> getHostIdToEndpoint()
     {
+        return getHostIdToEndpoint(false);
+    }
+
+    public Map<String, String> getHostIdToEndpointWithPort()
+    {
+        return getHostIdToEndpoint(true);
+    }
+
+    private Map<String, String> getHostIdToEndpoint(boolean withPort)
+    {
         Map<String, String> mapOut = new HashMap<>();
-        for (Map.Entry<InetAddress, UUID> entry : getTokenMetadata().getEndpointToHostIdMapForReading().entrySet())
-            mapOut.put(entry.getValue().toString(), entry.getKey().getHostAddress());
+        for (Map.Entry<InetAddressAndPort, UUID> entry : getTokenMetadata().getEndpointToHostIdMapForReading().entrySet())
+            mapOut.put(entry.getValue().toString(), entry.getKey().getHostAddress(withPort));
         return mapOut;
     }
 
@@ -1854,9 +1951,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param ranges
      * @return mapping of ranges to the replicas responsible for them.
     */
-    private Map<Range<Token>, List<InetAddress>> constructRangeToEndpointMap(String keyspace, List<Range<Token>> ranges)
+    private Map<Range<Token>, List<InetAddressAndPort>> constructRangeToEndpointMap(String keyspace, List<Range<Token>> ranges)
     {
-        Map<Range<Token>, List<InetAddress>> rangeToEndpointMap = new HashMap<>(ranges.size());
+        Map<Range<Token>, List<InetAddressAndPort>> rangeToEndpointMap = new HashMap<>(ranges.size());
         for (Range<Token> range : ranges)
         {
             rangeToEndpointMap.put(range, Keyspace.open(keyspace).getReplicationStrategy().getNaturalEndpoints(range.right));
@@ -1864,7 +1961,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return rangeToEndpointMap;
     }
 
-    public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue)
+    public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue)
     {
         // no-op
     }
@@ -1901,9 +1998,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * Note: Any time a node state changes from STATUS_NORMAL, it will not be visible to new nodes. So it follows that
      * you should never bootstrap a new node during a removenode, decommission or move.
      */
-    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value)
+    public void onChange(InetAddressAndPort endpoint, ApplicationState state, VersionedValue value)
     {
-        if (state == ApplicationState.STATUS)
+        if (state == ApplicationState.STATUS | state == ApplicationState.STATUS_WITH_PORT)
         {
             String[] pieces = splitValue(value);
             assert (pieces.length > 0);
@@ -1973,6 +2070,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                             throw new RuntimeException(e);
                         }
                         break;
+                    case NATIVE_ADDRESS_AND_PORT:
+                        try
+                        {
+                            InetAddressAndPort address = InetAddressAndPort.getByName(value.value);
+                            SystemKeyspace.updatePeerNativeAddress(endpoint, address);
+                        }
+                        catch (UnknownHostException e)
+                        {
+                            throw new RuntimeException(e);
+                        }
+                        break;
                     case SCHEMA:
                         SystemKeyspace.updatePeerInfo(endpoint, "schema_version", UUID.fromString(value.value));
                         MigrationManager.instance.scheduleSchemaPull(endpoint, epState);
@@ -1996,7 +2104,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return value.value.split(VersionedValue.DELIMITER_STR, -1);
     }
 
-    private void updateNetVersion(InetAddress endpoint, VersionedValue value)
+    private void updateNetVersion(InetAddressAndPort endpoint, VersionedValue value)
     {
         try
         {
@@ -2008,7 +2116,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
     }
 
-    public void updateTopology(InetAddress endpoint)
+    public void updateTopology(InetAddressAndPort endpoint)
     {
         if (getTokenMetadata().isMember(endpoint))
         {
@@ -2021,9 +2129,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         getTokenMetadata().updateTopology();
     }
 
-    private void updatePeerInfo(InetAddress endpoint)
+    private void updatePeerInfo(InetAddressAndPort endpoint)
     {
         EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
+        InetAddress native_address = null;
+        int native_port = DatabaseDescriptor.getNativeTransportPort();
+
         for (Map.Entry<ApplicationState, VersionedValue> entry : epState.states())
         {
             switch (entry.getKey())
@@ -2040,7 +2151,19 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 case RPC_ADDRESS:
                     try
                     {
-                        SystemKeyspace.updatePeerInfo(endpoint, "rpc_address", InetAddress.getByName(entry.getValue().value));
+                        native_address = InetAddress.getByName(entry.getValue().value);
+                    }
+                    catch (UnknownHostException e)
+                    {
+                        throw new RuntimeException(e);
+                    }
+                    break;
+                case NATIVE_ADDRESS_AND_PORT:
+                    try
+                    {
+                        InetAddressAndPort address = InetAddressAndPort.getByName(entry.getValue().value);
+                        native_address = address.address;
+                        native_port = address.port;
                     }
                     catch (UnknownHostException e)
                     {
@@ -2055,9 +2178,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     break;
             }
         }
+
+        //Some tests won't set all the states
+        if (native_address != null)
+        {
+            SystemKeyspace.updatePeerNativeAddress(endpoint,
+                                                   InetAddressAndPort.getByAddressOverrideDefaults(native_address,
+                                                                                                   native_port));
+        }
     }
 
-    private void notifyRpcChange(InetAddress endpoint, boolean ready)
+    private void notifyRpcChange(InetAddressAndPort endpoint, boolean ready)
     {
         if (ready)
             notifyUp(endpoint);
@@ -2065,7 +2196,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             notifyDown(endpoint);
     }
 
-    private void notifyUp(InetAddress endpoint)
+    private void notifyUp(InetAddressAndPort endpoint)
     {
         if (!isRpcReady(endpoint) || !Gossiper.instance.isAlive(endpoint))
             return;
@@ -2074,13 +2205,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             subscriber.onUp(endpoint);
     }
 
-    private void notifyDown(InetAddress endpoint)
+    private void notifyDown(InetAddressAndPort endpoint)
     {
         for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
             subscriber.onDown(endpoint);
     }
 
-    private void notifyJoined(InetAddress endpoint)
+    private void notifyJoined(InetAddressAndPort endpoint)
     {
         if (!isStatus(endpoint, VersionedValue.STATUS_NORMAL))
             return;
@@ -2089,25 +2220,25 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             subscriber.onJoinCluster(endpoint);
     }
 
-    private void notifyMoved(InetAddress endpoint)
+    private void notifyMoved(InetAddressAndPort endpoint)
     {
         for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
             subscriber.onMove(endpoint);
     }
 
-    private void notifyLeft(InetAddress endpoint)
+    private void notifyLeft(InetAddressAndPort endpoint)
     {
         for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
             subscriber.onLeaveCluster(endpoint);
     }
 
-    private boolean isStatus(InetAddress endpoint, String status)
+    private boolean isStatus(InetAddressAndPort endpoint, String status)
     {
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         return state != null && state.getStatus().equals(status);
     }
 
-    public boolean isRpcReady(InetAddress endpoint)
+    public boolean isRpcReady(InetAddressAndPort endpoint)
     {
         EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
         return state != null && state.isRpcReady();
@@ -2123,7 +2254,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public void setRpcReady(boolean value)
     {
-        EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(FBUtilities.getBroadcastAddress());
+        EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(FBUtilities.getBroadcastAddressAndPort());
         // if value is false we're OK with a null state, if it is true we are not.
         assert !value || state != null;
 
@@ -2131,7 +2262,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             Gossiper.instance.addLocalApplicationState(ApplicationState.RPC_READY, valueFactory.rpcReady(value));
     }
 
-    private Collection<Token> getTokensFor(InetAddress endpoint)
+    private Collection<Token> getTokensFor(InetAddressAndPort endpoint)
     {
         try
         {
@@ -2156,7 +2287,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      *
      * @param endpoint bootstrapping node
      */
-    private void handleStateBootstrap(InetAddress endpoint)
+    private void handleStateBootstrap(InetAddressAndPort endpoint)
     {
         Collection<Token> tokens;
         // explicitly check for TOKENS, because a bootstrapping node might be bootstrapping in legacy mode; that is, not using vnodes and no token specified
@@ -2186,12 +2317,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         tokenMetadata.updateHostId(Gossiper.instance.getHostId(endpoint), endpoint);
     }
 
-    private void handleStateBootreplacing(InetAddress newNode, String[] pieces)
+    private void handleStateBootreplacing(InetAddressAndPort newNode, String[] pieces)
     {
-        InetAddress oldNode;
+        InetAddressAndPort oldNode;
         try
         {
-            oldNode = InetAddress.getByName(pieces[1]);
+            oldNode = InetAddressAndPort.getByName(pieces[1]);
         }
         catch (Exception e)
         {
@@ -2204,7 +2335,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new RuntimeException(String.format("Node %s is trying to replace alive node %s.", newNode, oldNode));
         }
 
-        Optional<InetAddress> replacingNode = tokenMetadata.getReplacingNode(newNode);
+        Optional<InetAddressAndPort> replacingNode = tokenMetadata.getReplacingNode(newNode);
         if (replacingNode.isPresent() && !replacingNode.get().equals(oldNode))
         {
             throw new RuntimeException(String.format("Node %s is already replacing %s but is trying to replace %s.",
@@ -2228,12 +2359,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      *
      * @param endpoint node
      */
-    private void handleStateNormal(final InetAddress endpoint, final String status)
+    private void handleStateNormal(final InetAddressAndPort endpoint, final String status)
     {
         Collection<Token> tokens = getTokensFor(endpoint);
         Set<Token> tokensToUpdateInMetadata = new HashSet<>();
         Set<Token> tokensToUpdateInSystemKeyspace = new HashSet<>();
-        Set<InetAddress> endpointsToRemove = new HashSet<>();
+        Set<InetAddressAndPort> endpointsToRemove = new HashSet<>();
 
         if (logger.isDebugEnabled())
             logger.debug("Node {} state {}, token {}", endpoint, status, tokens);
@@ -2246,7 +2377,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                          endpoint,
                          Gossiper.instance.getEndpointStateForEndpoint(endpoint));
 
-        Optional<InetAddress> replacingNode = tokenMetadata.getReplacingNode(endpoint);
+        Optional<InetAddressAndPort> replacingNode = tokenMetadata.getReplacingNode(endpoint);
         if (replacingNode.isPresent())
         {
             assert !endpoint.equals(replacingNode.get()) : "Pending replacement endpoint with same address is not supported";
@@ -2259,7 +2390,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             endpointsToRemove.add(replacingNode.get());
         }
 
-        Optional<InetAddress> replacementNode = tokenMetadata.getReplacementNode(endpoint);
+        Optional<InetAddressAndPort> replacementNode = tokenMetadata.getReplacementNode(endpoint);
         if (replacementNode.isPresent())
         {
             logger.warn("Node {} is currently being replaced by node {}.", endpoint, replacementNode.get());
@@ -2268,7 +2399,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         updatePeerInfo(endpoint);
         // Order Matters, TM.updateHostID() should be called before TM.updateNormalToken(), (see CASSANDRA-4300).
         UUID hostId = Gossiper.instance.getHostId(endpoint);
-        InetAddress existing = tokenMetadata.getEndpointForHostId(hostId);
+        InetAddressAndPort existing = tokenMetadata.getEndpointForHostId(hostId);
         if (replacing && isReplacingSameAddress() && Gossiper.instance.getEndpointStateForEndpoint(DatabaseDescriptor.getReplaceAddress()) != null
             && (hostId.equals(Gossiper.instance.getHostId(DatabaseDescriptor.getReplaceAddress()))))
             logger.warn("Not updating token metadata for {} because I am replacing it", endpoint);
@@ -2276,7 +2407,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             if (existing != null && !existing.equals(endpoint))
             {
-                if (existing.equals(FBUtilities.getBroadcastAddress()))
+                if (existing.equals(FBUtilities.getBroadcastAddressAndPort()))
                 {
                     logger.warn("Not updating host ID {} for {} because it's mine", hostId, endpoint);
                     tokenMetadata.removeEndpoint(endpoint);
@@ -2303,7 +2434,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         for (final Token token : tokens)
         {
             // we don't want to update if this node is responsible for the token and it has a later startup time than endpoint.
-            InetAddress currentOwner = tokenMetadata.getEndpoint(token);
+            InetAddressAndPort currentOwner = tokenMetadata.getEndpoint(token);
             if (currentOwner == null)
             {
                 logger.debug("New node {} at token {}", endpoint, token);
@@ -2323,7 +2454,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
                 // currentOwner is no longer current, endpoint is.  Keep track of these moves, because when
                 // a host no longer has any tokens, we'll want to remove it.
-                Multimap<InetAddress, Token> epToTokenCopy = getTokenMetadata().getEndpointToTokenMapForReading();
+                Multimap<InetAddressAndPort, Token> epToTokenCopy = getTokenMetadata().getEndpointToTokenMapForReading();
                 epToTokenCopy.get(currentOwner).remove(token);
                 if (epToTokenCopy.get(currentOwner).size() < 1)
                     endpointsToRemove.add(currentOwner);
@@ -2348,7 +2479,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         boolean isMember = tokenMetadata.isMember(endpoint);
         boolean isMoving = tokenMetadata.isMoving(endpoint);
         tokenMetadata.updateNormalTokens(tokensToUpdateInMetadata, endpoint);
-        for (InetAddress ep : endpointsToRemove)
+        for (InetAddressAndPort ep : endpointsToRemove)
         {
             removeEndpoint(ep);
             if (replacing && DatabaseDescriptor.getReplaceAddress().equals(ep))
@@ -2375,7 +2506,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      *
      * @param endpoint node
      */
-    private void handleStateLeaving(InetAddress endpoint)
+    private void handleStateLeaving(InetAddressAndPort endpoint)
     {
         Collection<Token> tokens = getTokensFor(endpoint);
 
@@ -2408,7 +2539,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param endpoint If reason for leaving is decommission, endpoint is the leaving node.
      * @param pieces STATE_LEFT,token
      */
-    private void handleStateLeft(InetAddress endpoint, String[] pieces)
+    private void handleStateLeft(InetAddressAndPort endpoint, String[] pieces)
     {
         assert pieces.length >= 2;
         Collection<Token> tokens = getTokensFor(endpoint);
@@ -2425,7 +2556,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param endpoint moving endpoint address
      * @param pieces STATE_MOVING, token
      */
-    private void handleStateMoving(InetAddress endpoint, String[] pieces)
+    private void handleStateMoving(InetAddressAndPort endpoint, String[] pieces)
     {
         assert pieces.length >= 2;
         Token token = getTokenFactory().fromString(pieces[1]);
@@ -2444,11 +2575,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param endpoint node
      * @param pieces either REMOVED_TOKEN (node is gone) or REMOVING_TOKEN (replicas need to be restored)
      */
-    private void handleStateRemoving(InetAddress endpoint, String[] pieces)
+    private void handleStateRemoving(InetAddressAndPort endpoint, String[] pieces)
     {
         assert (pieces.length > 0);
 
-        if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
         {
             logger.info("Received removenode gossip about myself. Is this node rejoining after an explicit removenode?");
             try
@@ -2494,7 +2625,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
     }
 
-    private void excise(Collection<Token> tokens, InetAddress endpoint)
+    private void excise(Collection<Token> tokens, InetAddressAndPort endpoint)
     {
         logger.info("Removing tokens {} for {}", tokens, endpoint);
 
@@ -2510,20 +2641,20 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         PendingRangeCalculatorService.instance.update();
     }
 
-    private void excise(Collection<Token> tokens, InetAddress endpoint, long expireTime)
+    private void excise(Collection<Token> tokens, InetAddressAndPort endpoint, long expireTime)
     {
         addExpireTimeIfFound(endpoint, expireTime);
         excise(tokens, endpoint);
     }
 
     /** unlike excise we just need this endpoint gone without going through any notifications **/
-    private void removeEndpoint(InetAddress endpoint)
+    private void removeEndpoint(InetAddressAndPort endpoint)
     {
         Gossiper.instance.removeEndpoint(endpoint);
         SystemKeyspace.removeEndpoint(endpoint);
     }
 
-    protected void addExpireTimeIfFound(InetAddress endpoint, long expireTime)
+    protected void addExpireTimeIfFound(InetAddressAndPort endpoint, long expireTime)
     {
         if (expireTime != 0L)
         {
@@ -2543,23 +2674,23 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param ranges the ranges to find sources for
      * @return multimap of addresses to ranges the address is responsible for
      */
-    private Multimap<InetAddress, Range<Token>> getNewSourceRanges(String keyspaceName, Set<Range<Token>> ranges)
+    private Multimap<InetAddressAndPort, Range<Token>> getNewSourceRanges(String keyspaceName, Set<Range<Token>> ranges)
     {
-        InetAddress myAddress = FBUtilities.getBroadcastAddress();
-        Multimap<Range<Token>, InetAddress> rangeAddresses = Keyspace.open(keyspaceName).getReplicationStrategy().getRangeAddresses(tokenMetadata.cloneOnlyTokenMap());
-        Multimap<InetAddress, Range<Token>> sourceRanges = HashMultimap.create();
+        InetAddressAndPort myAddress = FBUtilities.getBroadcastAddressAndPort();
+        Multimap<Range<Token>, InetAddressAndPort> rangeAddresses = Keyspace.open(keyspaceName).getReplicationStrategy().getRangeAddresses(tokenMetadata.cloneOnlyTokenMap());
+        Multimap<InetAddressAndPort, Range<Token>> sourceRanges = HashMultimap.create();
         IFailureDetector failureDetector = FailureDetector.instance;
 
         // find alive sources for our new ranges
         for (Range<Token> range : ranges)
         {
-            Collection<InetAddress> possibleRanges = rangeAddresses.get(range);
+            Collection<InetAddressAndPort> possibleRanges = rangeAddresses.get(range);
             IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-            List<InetAddress> sources = snitch.getSortedListByProximity(myAddress, possibleRanges);
+            List<InetAddressAndPort> sources = snitch.getSortedListByProximity(myAddress, possibleRanges);
 
             assert (!sources.contains(myAddress));
 
-            for (InetAddress source : sources)
+            for (InetAddressAndPort source : sources)
             {
                 if (failureDetector.isAlive(source))
                 {
@@ -2576,7 +2707,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      *
      * @param remote node to send notification to
      */
-    private void sendReplicationNotification(InetAddress remote)
+    private void sendReplicationNotification(InetAddressAndPort remote)
     {
         // notify the remote token
         MessageOut msg = new MessageOut(MessagingService.Verb.REPLICATION_FINISHED);
@@ -2608,23 +2739,23 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      *
      * @param endpoint the node that left
      */
-    private void restoreReplicaCount(InetAddress endpoint, final InetAddress notifyEndpoint)
+    private void restoreReplicaCount(InetAddressAndPort endpoint, final InetAddressAndPort notifyEndpoint)
     {
-        Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> rangesToFetch = HashMultimap.create();
+        Multimap<String, Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> rangesToFetch = HashMultimap.create();
 
-        InetAddress myAddress = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort myAddress = FBUtilities.getBroadcastAddressAndPort();
 
         for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
         {
-            Multimap<Range<Token>, InetAddress> changedRanges = getChangedRangesForLeaving(keyspaceName, endpoint);
+            Multimap<Range<Token>, InetAddressAndPort> changedRanges = getChangedRangesForLeaving(keyspaceName, endpoint);
             Set<Range<Token>> myNewRanges = new HashSet<>();
-            for (Map.Entry<Range<Token>, InetAddress> entry : changedRanges.entries())
+            for (Map.Entry<Range<Token>, InetAddressAndPort> entry : changedRanges.entries())
             {
                 if (entry.getValue().equals(myAddress))
                     myNewRanges.add(entry.getKey());
             }
-            Multimap<InetAddress, Range<Token>> sourceRanges = getNewSourceRanges(keyspaceName, myNewRanges);
-            for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : sourceRanges.asMap().entrySet())
+            Multimap<InetAddressAndPort, Range<Token>> sourceRanges = getNewSourceRanges(keyspaceName, myNewRanges);
+            for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : sourceRanges.asMap().entrySet())
             {
                 rangesToFetch.put(keyspaceName, entry);
             }
@@ -2633,10 +2764,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         StreamPlan stream = new StreamPlan(StreamOperation.RESTORE_REPLICA_COUNT);
         for (String keyspaceName : rangesToFetch.keySet())
         {
-            for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : rangesToFetch.get(keyspaceName))
+            for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : rangesToFetch.get(keyspaceName))
             {
-                InetAddress source = entry.getKey();
-                InetAddress preferred = SystemKeyspace.getPreferredIP(source);
+                InetAddressAndPort source = entry.getKey();
+                InetAddressAndPort preferred = SystemKeyspace.getPreferredIP(source);
                 Collection<Range<Token>> ranges = entry.getValue();
                 if (logger.isDebugEnabled())
                     logger.debug("Requesting from {} ranges {}", source, StringUtils.join(ranges, ", "));
@@ -2661,7 +2792,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     }
 
     // needs to be modified to accept either a keyspace or ARS.
-    private Multimap<Range<Token>, InetAddress> getChangedRangesForLeaving(String keyspaceName, InetAddress endpoint)
+    private Multimap<Range<Token>, InetAddressAndPort> getChangedRangesForLeaving(String keyspaceName, InetAddressAndPort endpoint)
     {
         // First get all ranges the leaving endpoint is responsible for
         Collection<Range<Token>> ranges = getRangesForEndpoint(keyspaceName, endpoint);
@@ -2669,7 +2800,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (logger.isDebugEnabled())
             logger.debug("Node {} ranges [{}]", endpoint, StringUtils.join(ranges, ", "));
 
-        Map<Range<Token>, List<InetAddress>> currentReplicaEndpoints = new HashMap<>(ranges.size());
+        Map<Range<Token>, List<InetAddressAndPort>> currentReplicaEndpoints = new HashMap<>(ranges.size());
 
         // Find (for each range) all nodes that store replicas for these ranges as well
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap(); // don't do this in the loop! #7758
@@ -2683,7 +2814,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (temp.isMember(endpoint))
             temp.removeEndpoint(endpoint);
 
-        Multimap<Range<Token>, InetAddress> changedRanges = HashMultimap.create();
+        Multimap<Range<Token>, InetAddressAndPort> changedRanges = HashMultimap.create();
 
         // Go through the ranges and for each range check who will be
         // storing replicas for these ranges when the leaving endpoint
@@ -2692,7 +2823,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         // range.
         for (Range<Token> range : ranges)
         {
-            Collection<InetAddress> newReplicaEndpoints = Keyspace.open(keyspaceName).getReplicationStrategy().calculateNaturalEndpoints(range.right, temp);
+            Collection<InetAddressAndPort> newReplicaEndpoints = Keyspace.open(keyspaceName).getReplicationStrategy().calculateNaturalEndpoints(range.right, temp);
             newReplicaEndpoints.removeAll(currentReplicaEndpoints.get(range));
             if (logger.isDebugEnabled())
                 if (newReplicaEndpoints.isEmpty())
@@ -2705,7 +2836,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return changedRanges;
     }
 
-    public void onJoin(InetAddress endpoint, EndpointState epState)
+    public void onJoin(InetAddressAndPort endpoint, EndpointState epState)
     {
         for (Map.Entry<ApplicationState, VersionedValue> entry : epState.states())
         {
@@ -2714,7 +2845,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         MigrationManager.instance.scheduleSchemaPull(endpoint, epState);
     }
 
-    public void onAlive(InetAddress endpoint, EndpointState state)
+    public void onAlive(InetAddressAndPort endpoint, EndpointState state)
     {
         MigrationManager.instance.scheduleSchemaPull(endpoint, state);
 
@@ -2722,19 +2853,19 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             notifyUp(endpoint);
     }
 
-    public void onRemove(InetAddress endpoint)
+    public void onRemove(InetAddressAndPort endpoint)
     {
         tokenMetadata.removeEndpoint(endpoint);
         PendingRangeCalculatorService.instance.update();
     }
 
-    public void onDead(InetAddress endpoint, EndpointState state)
+    public void onDead(InetAddressAndPort endpoint, EndpointState state)
     {
         MessagingService.instance().convict(endpoint);
         notifyDown(endpoint);
     }
 
-    public void onRestart(InetAddress endpoint, EndpointState state)
+    public void onRestart(InetAddressAndPort endpoint, EndpointState state)
     {
         // If we have restarted before the node was even marked down, we need to reset the connection pool
         if (state.isAlive())
@@ -2753,15 +2884,25 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return FileUtils.stringifyFileSize(StorageMetrics.load.getCount());
     }
 
+    public Map<String, String> getLoadMapWithPort()
+    {
+        return getLoadMap(true);
+    }
+
     public Map<String, String> getLoadMap()
     {
+        return getLoadMap(false);
+    }
+
+    private Map<String, String> getLoadMap(boolean withPort)
+    {
         Map<String, String> map = new HashMap<>();
-        for (Map.Entry<InetAddress,Double> entry : LoadBroadcaster.instance.getLoadInfo().entrySet())
+        for (Map.Entry<InetAddressAndPort,Double> entry : LoadBroadcaster.instance.getLoadInfo().entrySet())
         {
-            map.put(entry.getKey().getHostAddress(), FileUtils.stringifyFileSize(entry.getValue()));
+            map.put(entry.getKey().getHostAddress(withPort), FileUtils.stringifyFileSize(entry.getValue()));
         }
         // gossiper doesn't see its own updates, so we need to special-case the local node
-        map.put(FBUtilities.getBroadcastAddress().getHostAddress(), getLoadString());
+        map.put(withPort ? FBUtilities.getJustBroadcastAddress().getHostAddress() : FBUtilities.getBroadcastAddressAndPort().toString(), getLoadString());
         return map;
     }
 
@@ -2779,13 +2920,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     }
 
     @Nullable
-    public InetAddress getEndpointForHostId(UUID hostId)
+    public InetAddressAndPort getEndpointForHostId(UUID hostId)
     {
         return tokenMetadata.getEndpointForHostId(hostId);
     }
 
     @Nullable
-    public UUID getHostIdForEndpoint(InetAddress address)
+    public UUID getHostIdForEndpoint(InetAddressAndPort address)
     {
         return tokenMetadata.getHostId(address);
     }
@@ -2794,15 +2935,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public List<String> getTokens()
     {
-        return getTokens(FBUtilities.getBroadcastAddress());
+        return getTokens(FBUtilities.getBroadcastAddressAndPort());
     }
 
     public List<String> getTokens(String endpoint) throws UnknownHostException
     {
-        return getTokens(InetAddress.getByName(endpoint));
+        return getTokens(InetAddressAndPort.getByName(endpoint));
     }
 
-    private List<String> getTokens(InetAddress endpoint)
+    private List<String> getTokens(InetAddressAndPort endpoint)
     {
         List<String> strTokens = new ArrayList<>();
         for (Token tok : getTokenMetadata().getTokens(endpoint))
@@ -2820,42 +2961,74 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return Schema.instance.getVersion().toString();
     }
 
+    @Deprecated
     public List<String> getLeavingNodes()
     {
-        return stringify(tokenMetadata.getLeavingEndpoints());
+        return stringify(tokenMetadata.getLeavingEndpoints(), false);
+    }
+
+    public List<String> getLeavingNodesWithPort()
+    {
+        return stringify(tokenMetadata.getLeavingEndpoints(), true);
     }
 
+    @Deprecated
     public List<String> getMovingNodes()
     {
         List<String> endpoints = new ArrayList<>();
 
-        for (Pair<Token, InetAddress> node : tokenMetadata.getMovingEndpoints())
+        for (Pair<Token, InetAddressAndPort> node : tokenMetadata.getMovingEndpoints())
+        {
+            endpoints.add(node.right.address.getHostAddress());
+        }
+
+        return endpoints;
+    }
+
+    public List<String> getMovingNodesWithPort()
+    {
+        List<String> endpoints = new ArrayList<>();
+
+        for (Pair<Token, InetAddressAndPort> node : tokenMetadata.getMovingEndpoints())
         {
-            endpoints.add(node.right.getHostAddress());
+            endpoints.add(node.right.toString());
         }
 
         return endpoints;
     }
 
+
     public List<String> getJoiningNodes()
     {
-        return stringify(tokenMetadata.getBootstrapTokens().valueSet());
+        return stringify(tokenMetadata.getBootstrapTokens().valueSet(), false);
+    }
+
+    @Deprecated
+    public List<String> getJoiningNodesWithPort()
+    {
+        return stringify(tokenMetadata.getBootstrapTokens().valueSet(), true);
     }
 
     public List<String> getLiveNodes()
     {
-        return stringify(Gossiper.instance.getLiveMembers());
+        return stringify(Gossiper.instance.getLiveMembers(), false);
+    }
+
+    @Deprecated
+    public List<String> getLiveNodesWithPort()
+    {
+        return stringify(Gossiper.instance.getLiveMembers(), true);
     }
 
-    public Set<InetAddress> getLiveRingMembers()
+    public Set<InetAddressAndPort> getLiveRingMembers()
     {
         return getLiveRingMembers(false);
     }
 
-    public Set<InetAddress> getLiveRingMembers(boolean excludeDeadStates)
+    public Set<InetAddressAndPort> getLiveRingMembers(boolean excludeDeadStates)
     {
-        Set<InetAddress> ret = new HashSet<>();
-        for (InetAddress ep : Gossiper.instance.getLiveMembers())
+        Set<InetAddressAndPort> ret = new HashSet<>();
+        for (InetAddressAndPort ep : Gossiper.instance.getLiveMembers())
         {
             if (excludeDeadStates)
             {
@@ -2871,9 +3044,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     }
 
 
+    @Deprecated
     public List<String> getUnreachableNodes()
     {
-        return stringify(Gossiper.instance.getUnreachableMembers());
+        return stringify(Gossiper.instance.getUnreachableMembers(), false);
+    }
+
+    public List<String> getUnreachableNodesWithPort()
+    {
+        return stringify(Gossiper.instance.getUnreachableMembers(), true);
     }
 
     public String[] getAllDataFileLocations()
@@ -2894,19 +3073,19 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return FileUtils.getCanonicalPath(DatabaseDescriptor.getSavedCachesLocation());
     }
 
-    private List<String> stringify(Iterable<InetAddress> endpoints)
+    private List<String> stringify(Iterable<InetAddressAndPort> endpoints, boolean withPort)
     {
         List<String> stringEndpoints = new ArrayList<>();
-        for (InetAddress ep : endpoints)
+        for (InetAddressAndPort ep : endpoints)
         {
-            stringEndpoints.add(ep.getHostAddress());
+            stringEndpoints.add(ep.getHostAddress(withPort));
         }
         return stringEndpoints;
     }
 
     public int getCurrentGenerationNumber()
     {
-        return Gossiper.instance.getCurrentGenerationNumber(FBUtilities.getBroadcastAddress());
+        return Gossiper.instance.getCurrentGenerationNumber(FBUtilities.getBroadcastAddressAndPort());
     }
 
     public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
@@ -3436,14 +3615,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param ep endpoint we are interested in.
      * @return primary ranges for the specified endpoint.
      */
-    public Collection<Range<Token>> getPrimaryRangesForEndpoint(String keyspace, InetAddress ep)
+    public Collection<Range<Token>> getPrimaryRangesForEndpoint(String keyspace, InetAddressAndPort ep)
     {
         AbstractReplicationStrategy strategy = Keyspace.open(keyspace).getReplicationStrategy();
         Collection<Range<Token>> primaryRanges = new HashSet<>();
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap();
         for (Token token : metadata.sortedTokens())
         {
-            List<InetAddress> endpoints = strategy.calculateNaturalEndpoints(token, metadata);
+            List<InetAddressAndPort> endpoints = strategy.calculateNaturalEndpoints(token, metadata);
             if (endpoints.size() > 0 && endpoints.get(0).equals(ep))
                 primaryRanges.add(new Range<>(metadata.getPredecessor(token), token));
         }
@@ -3453,23 +3632,23 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     /**
      * Get the "primary ranges" within local DC for the specified keyspace and endpoint.
      *
-     * @see #getPrimaryRangesForEndpoint(String, java.net.InetAddress)
+     * @see #getPrimaryRangesForEndpoint(String, InetAddressAndPort)
      * @param keyspace Keyspace name to check primary ranges
      * @param referenceEndpoint endpoint we are interested in.
      * @return primary ranges within local DC for the specified endpoint.
      */
-    public Collection<Range<Token>> getPrimaryRangeForEndpointWithinDC(String keyspace, InetAddress referenceEndpoint)
+    public Collection<Range<Token>> getPrimaryRangeForEndpointWithinDC(String keyspace, InetAddressAndPort referenceEndpoint)
     {
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap();
         String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(referenceEndpoint);
-        Collection<InetAddress> localDcNodes = metadata.getTopology().getDatacenterEndpoints().get(localDC);
+        Collection<InetAddressAndPort> localDcNodes = metadata.getTopology().getDatacenterEndpoints().get(localDC);
         AbstractReplicationStrategy strategy = Keyspace.open(keyspace).getReplicationStrategy();
 
         Collection<Range<Token>> localDCPrimaryRanges = new HashSet<>();
         for (Token token : metadata.sortedTokens())
         {
-            List<InetAddress> endpoints = strategy.calculateNaturalEndpoints(token, metadata);
-            for (InetAddress endpoint : endpoints)
+            List<InetAddressAndPort> endpoints = strategy.calculateNaturalEndpoints(token, metadata);
+            for (InetAddressAndPort endpoint : endpoints)
             {
                 if (localDcNodes.contains(endpoint))
                 {
@@ -3490,7 +3669,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param ep endpoint we are interested in.
      * @return ranges for the specified endpoint.
      */
-    Collection<Range<Token>> getRangesForEndpoint(String keyspaceName, InetAddress ep)
+    Collection<Range<Token>> getRangesForEndpoint(String keyspaceName, InetAddressAndPort ep)
     {
         return Keyspace.open(keyspaceName).getReplicationStrategy().getAddressRanges().get(ep);
     }
@@ -3530,6 +3709,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param key key for which we need to find the endpoint
      * @return the endpoint responsible for this key
      */
+    @Deprecated
     public List<InetAddress> getNaturalEndpoints(String keyspaceName, String cf, String key)
     {
         KeyspaceMetadata ksMetaData = Schema.instance.getKeyspaceMetadata(keyspaceName);
@@ -3540,12 +3720,32 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (metadata == null)
             throw new IllegalArgumentException("Unknown table '" + cf + "' in keyspace '" + keyspaceName + "'");
 
-        return getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(metadata.partitionKeyType.fromString(key)));
+        return getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(metadata.partitionKeyType.fromString(key))).stream().map(i -> i.address).collect(toList());
+    }
+
+    public List<String> getNaturalEndpointsWithPort(String keyspaceName, String cf, String key)
+    {
+        KeyspaceMetadata ksMetaData = Schema.instance.getKeyspaceMetadata(keyspaceName);
+        if (ksMetaData == null)
+            throw new IllegalArgumentException("Unknown keyspace '" + keyspaceName + "'");
+
+        TableMetadata metadata = ksMetaData.getTableOrViewNullable(cf);
+        if (metadata == null)
+            throw new IllegalArgumentException("Unknown table '" + cf + "' in keyspace '" + keyspaceName + "'");
+
+        return stringify(getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(metadata.partitionKeyType.fromString(key))), true);
     }
 
+
+    @Deprecated
     public List<InetAddress> getNaturalEndpoints(String keyspaceName, ByteBuffer key)
     {
-        return getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(key));
+        return getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(key)).stream().map(i -> i.address).collect(toList());
+    }
+
+    public List<String> getNaturalEndpointsWithPort(String keyspaceName, ByteBuffer key)
+    {
+        return stringify(getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(key)), true);
     }
 
     /**
@@ -3556,7 +3756,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param pos position for which we need to find the endpoint
      * @return the endpoint responsible for this token
      */
-    public List<InetAddress> getNaturalEndpoints(String keyspaceName, RingPosition pos)
+    public List<InetAddressAndPort> getNaturalEndpoints(String keyspaceName, RingPosition pos)
     {
         return Keyspace.open(keyspaceName).getReplicationStrategy().getNaturalEndpoints(pos);
     }
@@ -3564,7 +3764,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     /**
      * Returns the endpoints currently responsible for storing the token plus pending ones
      */
-    public Iterable<InetAddress> getNaturalAndPendingEndpoints(String keyspaceName, Token token)
+    public Iterable<InetAddressAndPort> getNaturalAndPendingEndpoints(String keyspaceName, Token token)
     {
         return Iterables.concat(getNaturalEndpoints(keyspaceName, token), tokenMetadata.pendingEndpointsFor(token, keyspaceName));
     }
@@ -3577,14 +3777,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param key key for which we need to find the endpoint
      * @return the endpoint responsible for this key
      */
-    public List<InetAddress> getLiveNaturalEndpoints(Keyspace keyspace, ByteBuffer key)
+    public List<InetAddressAndPort> getLiveNaturalEndpoints(Keyspace keyspace, ByteBuffer key)
     {
         return getLiveNaturalEndpoints(keyspace, tokenMetadata.decorateKey(key));
     }
 
-    public List<InetAddress> getLiveNaturalEndpoints(Keyspace keyspace, RingPosition pos)
+    public List<InetAddressAndPort> getLiveNaturalEndpoints(Keyspace keyspace, RingPosition pos)
     {
-        List<InetAddress> liveEps = new ArrayList<>();
+        List<InetAddressAndPort> liveEps = new ArrayList<>();
         getLiveNaturalEndpoints(keyspace, pos, liveEps);
         return liveEps;
     }
@@ -3597,11 +3797,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param pos position for which we need to find the endpoint
      * @param liveEps the list of endpoints to mutate
      */
-    public void getLiveNaturalEndpoints(Keyspace keyspace, RingPosition pos, List<InetAddress> liveEps)
+    public void getLiveNaturalEndpoints(Keyspace keyspace, RingPosition pos, List<InetAddressAndPort> liveEps)
     {
-        List<InetAddress> endpoints = keyspace.getReplicationStrategy().getNaturalEndpoints(pos);
+        List<InetAddressAndPort> endpoints = keyspace.getReplicationStrategy().getNaturalEndpoints(pos);
 
-        for (InetAddress endpoint : endpoints)
+        for (InetAddressAndPort endpoint : endpoints)
         {
             if (FailureDetector.instance.isAlive(endpoint))
                 liveEps.add(endpoint);
@@ -3718,8 +3918,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     private void startLeaving()
     {
+        Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS_WITH_PORT, valueFactory.leaving(getLocalTokens()));
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.leaving(getLocalTokens()));
-        tokenMetadata.addLeavingEndpoint(FBUtilities.getBroadcastAddress());
+        tokenMetadata.addLeavingEndpoint(FBUtilities.getBroadcastAddressAndPort());
         PendingRangeCalculatorService.instance.update();
     }
 
@@ -3728,7 +3929,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         TokenMetadata metadata = tokenMetadata.cloneAfterAllLeft();
         if (operationMode != Mode.LEAVING)
         {
-            if (!tokenMetadata.isMember(FBUtilities.getBroadcastAddress()))
+            if (!tokenMetadata.isMember(FBUtilities.getBroadcastAddressAndPort()))
                 throw new UnsupportedOperationException("local node is not a member of the token ring yet");
             if (metadata.getAllEndpoints().size() < 2)
                     throw new UnsupportedOperationException("no other normal nodes in the ring; decommission would be pointless");
@@ -3745,7 +3946,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             PendingRangeCalculatorService.instance.blockUntilFinished();
 
-            String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+            String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
 
             if (operationMode != Mode.LEAVING) // If we're already decommissioning there is no point checking RF/pending ranges
             {
@@ -3772,7 +3973,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                                                     + keyspaceName + " (RF = " + rf + ", N = " + numNodes + ")."
                                                                     + " Perform a forceful decommission to ignore.");
                     }
-                    if (tokenMetadata.getPendingRanges(keyspaceName, FBUtilities.getBroadcastAddress()).size() > 0)
+                    if (tokenMetadata.getPendingRanges(keyspaceName, FBUtilities.getBroadcastAddressAndPort()).size() > 0)
                         throw new UnsupportedOperationException("data is currently moving to this node; unable to leave the ring");
                 }
             }
@@ -3822,9 +4023,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     private void leaveRing()
     {
         SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.NEEDS_BOOTSTRAP);
-        tokenMetadata.removeEndpoint(FBUtilities.getBroadcastAddress());
+        tokenMetadata.removeEndpoint(FBUtilities.getBroadcastAddressAndPort());
         PendingRangeCalculatorService.instance.update();
 
+        Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS_WITH_PORT, valueFactory.left(getLocalTokens(),Gossiper.computeExpireTime()));
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.left(getLocalTokens(),Gossiper.computeExpireTime()));
         int delay = Math.max(RING_DELAY, Gossiper.intervalInMillis * 2);
         logger.info("Announcing that I have left the ring for {}ms", delay);
@@ -3833,11 +4035,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     private void unbootstrap(Runnable onFinish) throws ExecutionException, InterruptedException
     {
-        Map<String, Multimap<Range<Token>, InetAddress>> rangesToStream = new HashMap<>();
+        Map<String, Multimap<Range<Token>, InetAddressAndPort>> rangesToStream = new HashMap<>();
 
         for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
         {
-            Multimap<Range<Token>, InetAddress> rangesMM = getChangedRangesForLeaving(keyspaceName, FBUtilities.getBroadcastAddress());
+            Multimap<Range<Token>, InetAddressAndPort> rangesMM = getChangedRangesForLeaving(keyspaceName, FBUtilities.getBroadcastAddressAndPort());
 
             if (logger.isDebugEnabled())
                 logger.debug("Ranges needing transfer are [{}]", StringUtils.join(rangesMM.keySet(), ","));
@@ -3878,11 +4080,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     private UUID getPreferredHintsStreamTarget()
     {
-        List<InetAddress> candidates = new ArrayList<>(StorageService.instance.getTokenMetadata().cloneAfterAllLeft().getAllEndpoints());
-     

<TRUNCATED>

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


[03/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 4d9829f..93a1c20 100644
--- a/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
+++ b/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java
@@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.ArrayList;
 
+import com.google.common.net.InetAddresses;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -39,6 +40,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage;
@@ -49,7 +51,7 @@ 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 InetAddressAndPort addr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 0);
     private static final int MESSAGING_VERSION = MessagingService.current_version;
     private static final int VERSION_30 = MessagingService.VERSION_30;
 
@@ -84,7 +86,7 @@ public class InboundHandshakeHandlerTest
     {
         handler = new InboundHandshakeHandler(new TestAuthenticator(true));
         channel = new EmbeddedChannel(handler);
-        boolean result = handler.handleAuthenticate(addr, channel.pipeline().firstContext());
+        boolean result = handler.handleAuthenticate(new InetSocketAddress(addr.address, addr.port), channel.pipeline().firstContext());
         Assert.assertTrue(result);
         Assert.assertTrue(channel.isOpen());
     }
@@ -92,7 +94,7 @@ public class InboundHandshakeHandlerTest
     @Test
     public void handleAuthenticate_Bad()
     {
-        boolean result = handler.handleAuthenticate(addr, channel.pipeline().firstContext());
+        boolean result = handler.handleAuthenticate(new InetSocketAddress(addr.address, addr.port), channel.pipeline().firstContext());
         Assert.assertFalse(result);
         Assert.assertFalse(channel.isOpen());
         Assert.assertFalse(channel.isActive());
@@ -178,7 +180,7 @@ public class InboundHandshakeHandlerTest
         if (buf.refCnt() > 0)
             buf.release();
 
-        buf = new ThirdHandshakeMessage(MESSAGING_VERSION, addr.getAddress()).encode(PooledByteBufAllocator.DEFAULT);
+        buf = new ThirdHandshakeMessage(MESSAGING_VERSION, addr).encode(PooledByteBufAllocator.DEFAULT);
         state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
 
         Assert.assertEquals(State.HANDSHAKE_COMPLETE, state);
@@ -203,7 +205,7 @@ public class InboundHandshakeHandlerTest
     {
         buf = Unpooled.buffer(32, 32);
         buf.writeInt(MESSAGING_VERSION + 1);
-        CompactEndpointSerializationHelper.serialize(addr.getAddress(), new ByteBufOutputStream(buf));
+        CompactEndpointSerializationHelper.instance.serialize(addr, new ByteBufDataOutputPlus(buf), MESSAGING_VERSION + 1);
         State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
         Assert.assertEquals(State.HANDSHAKE_FAIL, state);
         Assert.assertFalse(channel.isOpen());
@@ -215,7 +217,7 @@ public class InboundHandshakeHandlerTest
     {
         buf = Unpooled.buffer(32, 32);
         buf.writeInt(MESSAGING_VERSION);
-        CompactEndpointSerializationHelper.serialize(addr.getAddress(), new ByteBufOutputStream(buf));
+        CompactEndpointSerializationHelper.instance.serialize(addr, new ByteBufDataOutputPlus(buf), MESSAGING_VERSION);
         State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf);
         Assert.assertEquals(State.HANDSHAKE_COMPLETE, state);
         Assert.assertTrue(channel.isOpen());
@@ -228,7 +230,7 @@ public class InboundHandshakeHandlerTest
         ChannelPipeline pipeline = channel.pipeline();
         Assert.assertNotNull(pipeline.get(InboundHandshakeHandler.class));
 
-        handler.setupMessagingPipeline(pipeline, addr.getAddress(), false, MESSAGING_VERSION);
+        handler.setupMessagingPipeline(pipeline, addr, false, MESSAGING_VERSION);
         Assert.assertNotNull(pipeline.get(MessageInHandler.class));
         Assert.assertNull(pipeline.get(Lz4FrameDecoder.class));
         Assert.assertNull(pipeline.get(Lz4FrameEncoder.class));
@@ -241,7 +243,7 @@ public class InboundHandshakeHandlerTest
         ChannelPipeline pipeline = channel.pipeline();
         Assert.assertNotNull(pipeline.get(InboundHandshakeHandler.class));
 
-        handler.setupMessagingPipeline(pipeline, addr.getAddress(), true, MESSAGING_VERSION);
+        handler.setupMessagingPipeline(pipeline, addr, true, MESSAGING_VERSION);
         Assert.assertNotNull(pipeline.get(MessageInHandler.class));
         Assert.assertNotNull(pipeline.get(Lz4FrameDecoder.class));
         Assert.assertNull(pipeline.get(Lz4FrameEncoder.class));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index bb82d2c..43cb964 100644
--- a/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java
+++ b/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java
@@ -20,40 +20,36 @@ 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.UUID;
 import java.util.function.BiConsumer;
 
-import com.google.common.base.Charsets;
+import com.google.common.net.InetAddresses;
 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.ParameterType;
 import org.apache.cassandra.net.async.MessageInHandler.MessageHeader;
-import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis;
+import org.apache.cassandra.utils.UUIDGen;
 
 public class MessageInHandlerTest
 {
-    private static final InetSocketAddress addr = new InetSocketAddress("127.0.0.1", 0);
+    private static final InetAddressAndPort addr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 0);
     private static final int MSG_VERSION = MessagingService.current_version;
 
     private static final int MSG_ID = 42;
@@ -81,7 +77,7 @@ public class MessageInHandlerTest
         buf.writeInt(-1);
         buf.writerIndex(len);
 
-        MessageInHandler handler = new MessageInHandler(addr.getAddress(), MSG_VERSION, null);
+        MessageInHandler handler = new MessageInHandler(addr, MSG_VERSION, null);
         EmbeddedChannel channel = new EmbeddedChannel(handler);
         Assert.assertTrue(channel.isOpen());
         channel.writeInbound(buf);
@@ -98,22 +94,25 @@ public class MessageInHandlerTest
     @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));
+        UUID uuid = UUIDGen.getTimeUUID();
+        Map<ParameterType, Object> parameters = new HashMap<>();
+        parameters.put(ParameterType.FAILURE_REASON, (short)42);
+        parameters.put(ParameterType.TRACE_SESSION, uuid);
         MessageInWrapper result = decode_HappyPath(parameters);
         Assert.assertEquals(2, result.messageIn.parameters.size());
+        Assert.assertEquals((short)42, result.messageIn.parameters.get(ParameterType.FAILURE_REASON));
+        Assert.assertEquals(uuid, result.messageIn.parameters.get(ParameterType.TRACE_SESSION));
     }
 
-    private MessageInWrapper decode_HappyPath(Map<String, byte[]> parameters) throws Exception
+    private MessageInWrapper decode_HappyPath(Map<ParameterType, Object> parameters) throws Exception
     {
         MessageOut msgOut = new MessageOut(MessagingService.Verb.ECHO);
-        for (Map.Entry<String, byte[]> param : parameters.entrySet())
+        for (Map.Entry<ParameterType, Object> 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);
+        MessageInHandler handler = new MessageInHandler(addr, MSG_VERSION, wrapper.messageConsumer);
         List<Object> out = new ArrayList<>();
         handler.decode(null, buf, out);
 
@@ -140,7 +139,7 @@ public class MessageInHandlerTest
     public void decode_WithHalfReceivedParameters() throws Exception
     {
         MessageOut msgOut = new MessageOut(MessagingService.Verb.ECHO);
-        msgOut = msgOut.withParameter("p3", "val1".getBytes(Charsets.UTF_8));
+        msgOut = msgOut.withParameter(ParameterType.FAILURE_REASON, (short)42);
 
         serialize(msgOut);
 
@@ -150,7 +149,7 @@ public class MessageInHandlerTest
         buf.writerIndex(originalWriterIndex - 6);
 
         MessageInWrapper wrapper = new MessageInWrapper();
-        MessageInHandler handler = new MessageInHandler(addr.getAddress(), MSG_VERSION, wrapper.messageConsumer);
+        MessageInHandler handler = new MessageInHandler(addr, MSG_VERSION, wrapper.messageConsumer);
         List<Object> out = new ArrayList<>();
         handler.decode(null, buf, out);
 
@@ -221,7 +220,7 @@ public class MessageInHandlerTest
     @Test
     public void exceptionHandled()
     {
-        MessageInHandler handler = new MessageInHandler(addr.getAddress(), MSG_VERSION, null);
+        MessageInHandler handler = new MessageInHandler(addr, MSG_VERSION, null);
         EmbeddedChannel channel = new EmbeddedChannel(handler);
         Assert.assertTrue(channel.isOpen());
         handler.exceptionCaught(channel.pipeline().firstContext(), new EOFException());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index 566dfdb..86112ae 100644
--- a/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java
+++ b/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java
@@ -44,8 +44,10 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.ParameterType;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.UUIDGen;
 
@@ -65,15 +67,15 @@ public class MessageOutHandlerTest
     }
 
     @Before
-    public void setup()
+    public void setup() throws Exception
     {
         setup(MessageOutHandler.AUTO_FLUSH_THRESHOLD);
     }
 
-    private void setup(int flushThreshold)
+    private void setup(int flushThreshold) throws Exception
     {
-        OutboundConnectionIdentifier connectionId = OutboundConnectionIdentifier.small(new InetSocketAddress("127.0.0.1", 0),
-                                                                                       new InetSocketAddress("127.0.0.2", 0));
+        OutboundConnectionIdentifier connectionId = OutboundConnectionIdentifier.small(InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", 0),
+                                                                                       InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", 0));
         OutboundMessagingConnection omc = new NonSendingOutboundMessagingConnection(connectionId, null, Optional.empty());
         channel = new EmbeddedChannel();
         channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty());
@@ -91,7 +93,7 @@ public class MessageOutHandlerTest
     }
 
     @Test
-    public void write_WithFlush() throws ExecutionException, InterruptedException, TimeoutException
+    public void write_WithFlush() throws Exception
     {
         setup(1);
         MessageOut message = new MessageOut(MessagingService.Verb.ECHO);
@@ -217,7 +219,7 @@ public class MessageOutHandlerTest
     public void captureTracingInfo_ForceException()
     {
         MessageOut message = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
-                             .withParameter(Tracing.TRACE_HEADER, new byte[9]);
+                             .withParameter(ParameterType.TRACE_SESSION, new byte[9]);
         handler.captureTracingInfo(new QueuedMessage(message, 42));
     }
 
@@ -226,7 +228,7 @@ public class MessageOutHandlerTest
     {
         UUID uuid = UUID.randomUUID();
         MessageOut message = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
-                             .withParameter(Tracing.TRACE_HEADER, UUIDGen.decompose(uuid));
+                             .withParameter(ParameterType.TRACE_SESSION, uuid);
         handler.captureTracingInfo(new QueuedMessage(message, 42));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index 0550490..4607d5c 100644
--- a/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java
+++ b/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.net.async;
 
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.Optional;
 
@@ -47,6 +46,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.NettyFactory.InboundInitializer;
 import org.apache.cassandra.net.async.NettyFactory.OutboundInitializer;
@@ -56,8 +56,8 @@ 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 InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9876);
+    private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 9876);
     private static final int receiveBufferSize = 1 << 16;
     private static final IInternodeAuthenticator AUTHENTICATOR = new AllowAllInternodeAuthenticator();
     private static final boolean EPOLL_AVAILABLE = NativeTransportService.useEpoll();
@@ -129,7 +129,6 @@ public class NettyFactoryTest
         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);
@@ -144,7 +143,7 @@ public class NettyFactoryTest
     @Test(expected = ConfigurationException.class)
     public void createServerChannel_UnbindableAddress()
     {
-        InetSocketAddress addr = new InetSocketAddress("1.1.1.1", 9876);
+        InetAddressAndPort addr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("1.1.1.1"), 9876);
         InboundInitializer inboundInitializer = new InboundInitializer(AUTHENTICATOR, null, channelGroup);
         NettyFactory.instance.createInboundChannel(addr, inboundInitializer, receiveBufferSize);
     }
@@ -162,10 +161,10 @@ public class NettyFactoryTest
         Assert.assertEquals(2, NettyFactory.determineAcceptGroupSize(serverEncryptionOptions));
         serverEncryptionOptions.enable_legacy_ssl_storage_port = false;
 
-        InetAddress originalBroadcastAddr = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort originalBroadcastAddr = FBUtilities.getBroadcastAddressAndPort();
         try
         {
-            FBUtilities.setBroadcastInetAddress(InetAddresses.increment(FBUtilities.getLocalAddress()));
+            FBUtilities.setBroadcastInetAddress(InetAddresses.increment(FBUtilities.getLocalAddressAndPort().address));
             DatabaseDescriptor.setListenOnBroadcastAddress(true);
 
             serverEncryptionOptions.enabled = false;
@@ -178,7 +177,7 @@ public class NettyFactoryTest
         }
         finally
         {
-            FBUtilities.setBroadcastInetAddress(originalBroadcastAddr);
+            FBUtilities.setBroadcastInetAddress(originalBroadcastAddr.address);
             DatabaseDescriptor.setListenOnBroadcastAddress(false);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index f8bfab1..be71fd4 100644
--- a/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java
+++ b/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java
@@ -23,6 +23,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Optional;
 
+import com.google.common.net.InetAddresses;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -38,6 +39,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage;
 import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult;
@@ -47,8 +49,8 @@ import static org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeR
 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 InetAddressAndPort localAddr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 0);
+    private static final InetAddressAndPort remoteAddr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0);
     private static final String HANDLER_NAME = "clientHandshakeHandler";
 
     private EmbeddedChannel channel;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 641c28c..bf6e066 100644
--- a/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
+++ b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.net.ssl.SSLHandshakeException;
 
+import com.google.common.net.InetAddresses;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -45,6 +46,7 @@ import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.AbstractEndpointSnitch;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.MessagingServiceTest;
@@ -59,9 +61,9 @@ import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.R
 
 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 InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9998);
+    private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 9999);
+    private static final InetAddressAndPort RECONNECT_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.3"), 9999);
     private static final int MESSAGING_VERSION = MessagingService.current_version;
 
     private OutboundConnectionIdentifier connectionId;
@@ -131,47 +133,47 @@ public class OutboundMessagingConnectionTest
     public void shouldCompressConnection_None()
     {
         DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.none);
-        Assert.assertFalse(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR.getAddress(), REMOTE_ADDR.getAddress()));
+        Assert.assertFalse(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR));
     }
 
     @Test
     public void shouldCompressConnection_All()
     {
         DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.all);
-        Assert.assertTrue(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR.getAddress(), REMOTE_ADDR.getAddress()));
+        Assert.assertTrue(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR));
     }
 
     @Test
     public void shouldCompressConnection_SameDc()
     {
         TestSnitch snitch = new TestSnitch();
-        snitch.add(LOCAL_ADDR.getAddress(), "dc1");
-        snitch.add(REMOTE_ADDR.getAddress(), "dc1");
+        snitch.add(LOCAL_ADDR, "dc1");
+        snitch.add(REMOTE_ADDR, "dc1");
         DatabaseDescriptor.setEndpointSnitch(snitch);
         DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc);
-        Assert.assertFalse(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR.getAddress(), REMOTE_ADDR.getAddress()));
+        Assert.assertFalse(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR));
     }
 
     private static class TestSnitch extends AbstractEndpointSnitch
     {
-        private Map<InetAddress, String> nodeToDc = new HashMap<>();
+        private Map<InetAddressAndPort, String> nodeToDc = new HashMap<>();
 
-        void add(InetAddress node, String dc)
+        void add(InetAddressAndPort node, String dc)
         {
             nodeToDc.put(node, dc);
         }
 
-        public String getRack(InetAddress endpoint)
+        public String getRack(InetAddressAndPort endpoint)
         {
             return null;
         }
 
-        public String getDatacenter(InetAddress endpoint)
+        public String getDatacenter(InetAddressAndPort endpoint)
         {
             return nodeToDc.get(endpoint);
         }
 
-        public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+        public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
         {
             return 0;
         }
@@ -181,11 +183,11 @@ public class OutboundMessagingConnectionTest
     public void shouldCompressConnection_DifferentDc()
     {
         TestSnitch snitch = new TestSnitch();
-        snitch.add(LOCAL_ADDR.getAddress(), "dc1");
-        snitch.add(REMOTE_ADDR.getAddress(), "dc2");
+        snitch.add(LOCAL_ADDR, "dc1");
+        snitch.add(REMOTE_ADDR, "dc2");
         DatabaseDescriptor.setEndpointSnitch(snitch);
         DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc);
-        Assert.assertTrue(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR.getAddress(), REMOTE_ADDR.getAddress()));
+        Assert.assertTrue(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR));
     }
 
     @Test
@@ -247,7 +249,7 @@ public class OutboundMessagingConnectionTest
 
         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);
+                                                               new MessagingServiceTest.MockBackPressureStrategy(null).newState(REMOTE_ADDR), auth);
         omc = pool.getConnection(messageOut);
         Assert.assertSame(State.NOT_READY, omc.getState());
         Assert.assertFalse(omc.connect());
@@ -371,7 +373,7 @@ public class OutboundMessagingConnectionTest
         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.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR));
         Assert.assertNull(omc.getConnectionTimeoutFuture());
         Assert.assertTrue(connectionTimeoutFuture.isCancelled());
     }
@@ -391,7 +393,7 @@ public class OutboundMessagingConnectionTest
         Assert.assertTrue(channelWriter.isClosed());
         Assert.assertNull(omc.getChannelWriter());
         Assert.assertEquals(CLOSED, omc.getState());
-        Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR.getAddress()));
+        Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR));
         Assert.assertNull(omc.getConnectionTimeoutFuture());
         Assert.assertTrue(connectionTimeoutFuture.isCancelled());
     }
@@ -408,7 +410,7 @@ public class OutboundMessagingConnectionTest
         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(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR));
         Assert.assertEquals(count, omc.backlogSize());
     }
 
@@ -423,7 +425,7 @@ public class OutboundMessagingConnectionTest
         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(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR));
         Assert.assertEquals(0, omc.backlogSize());
     }
 
@@ -512,8 +514,8 @@ public class OutboundMessagingConnectionTest
         OutboundConnectionIdentifier connectionId = omc.getConnectionId();
         omc.maybeUpdateConnectionId();
         Assert.assertNotEquals(connectionId, omc.getConnectionId());
-        Assert.assertEquals(new InetSocketAddress(REMOTE_ADDR.getAddress(), DatabaseDescriptor.getSSLStoragePort()), omc.getConnectionId().remoteAddress());
-        Assert.assertEquals(new InetSocketAddress(REMOTE_ADDR.getAddress(), DatabaseDescriptor.getSSLStoragePort()), omc.getConnectionId().connectionAddress());
+        Assert.assertEquals(InetAddressAndPort.getByAddressOverrideDefaults(REMOTE_ADDR.address, DatabaseDescriptor.getSSLStoragePort()), omc.getConnectionId().remote());
+        Assert.assertEquals(InetAddressAndPort.getByAddressOverrideDefaults(REMOTE_ADDR.address, DatabaseDescriptor.getSSLStoragePort()), omc.getConnectionId().connectionAddress());
         Assert.assertEquals(peerVersion, omc.getTargetVersion());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index 655cd15..ecd8697 100644
--- a/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java
+++ b/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java
@@ -18,10 +18,10 @@
 
 package org.apache.cassandra.net.async;
 
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.google.common.net.InetAddresses;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -35,6 +35,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.BackPressureState;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
@@ -42,9 +43,9 @@ import org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionTyp
 
 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 InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9476);
+    private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 9476);
+    private static final InetAddressAndPort RECONNECT_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("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); }};
 
@@ -59,7 +60,7 @@ public class OutboundMessagingPoolTest
     @Before
     public void setup()
     {
-        BackPressureState backPressureState = DatabaseDescriptor.getBackPressureStrategy().newState(REMOTE_ADDR.getAddress());
+        BackPressureState backPressureState = DatabaseDescriptor.getBackPressureStrategy().newState(REMOTE_ADDR);
         pool = new OutboundMessagingPool(REMOTE_ADDR, LOCAL_ADDR, null, backPressureState, new AllowAllInternodeAuthenticator());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/AbstractRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/AbstractRepairTest.java b/test/unit/org/apache/cassandra/repair/AbstractRepairTest.java
index 21c51c6..970e648 100644
--- a/test/unit/org/apache/cassandra/repair/AbstractRepairTest.java
+++ b/test/unit/org/apache/cassandra/repair/AbstractRepairTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Set;
 import java.util.UUID;
@@ -33,6 +32,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -41,19 +41,19 @@ import org.apache.cassandra.utils.UUIDGen;
 @Ignore
 public abstract class AbstractRepairTest
 {
-    protected static final InetAddress COORDINATOR;
-    protected static final InetAddress PARTICIPANT1;
-    protected static final InetAddress PARTICIPANT2;
-    protected static final InetAddress PARTICIPANT3;
+    protected static final InetAddressAndPort COORDINATOR;
+    protected static final InetAddressAndPort PARTICIPANT1;
+    protected static final InetAddressAndPort PARTICIPANT2;
+    protected static final InetAddressAndPort PARTICIPANT3;
 
     static
     {
         try
         {
-            COORDINATOR = InetAddress.getByName("10.0.0.1");
-            PARTICIPANT1 = InetAddress.getByName("10.0.0.1");
-            PARTICIPANT2 = InetAddress.getByName("10.0.0.2");
-            PARTICIPANT3 = InetAddress.getByName("10.0.0.3");
+            COORDINATOR = InetAddressAndPort.getByName("10.0.0.1");
+            PARTICIPANT1 = InetAddressAndPort.getByName("10.0.0.1");
+            PARTICIPANT2 = InetAddressAndPort.getByName("10.0.0.2");
+            PARTICIPANT3 = InetAddressAndPort.getByName("10.0.0.3");
         }
         catch (UnknownHostException e)
         {
@@ -64,7 +64,7 @@ public abstract class AbstractRepairTest
         DatabaseDescriptor.daemonInitialization();
     }
 
-    protected static final Set<InetAddress> PARTICIPANTS = ImmutableSet.of(PARTICIPANT1, PARTICIPANT2, PARTICIPANT3);
+    protected static final Set<InetAddressAndPort> PARTICIPANTS = ImmutableSet.of(PARTICIPANT1, PARTICIPANT2, PARTICIPANT3);
 
     protected static Token t(int v)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
index 7f3dbff..95046bd 100644
--- a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
+++ b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
@@ -36,6 +35,7 @@ import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableId;
@@ -78,8 +78,8 @@ public class LocalSyncTaskTest extends AbstractRepairTest
     @Test
     public void testNoDifference() throws Throwable
     {
-        final InetAddress ep1 = InetAddress.getByName("127.0.0.1");
-        final InetAddress ep2 = InetAddress.getByName("127.0.0.1");
+        final InetAddressAndPort ep1 = InetAddressAndPort.getByName("127.0.0.1");
+        final InetAddressAndPort ep2 = InetAddressAndPort.getByName("127.0.0.1");
 
         Range<Token> range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken());
         RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), KEYSPACE1, "Standard1", Arrays.asList(range));
@@ -106,7 +106,7 @@ public class LocalSyncTaskTest extends AbstractRepairTest
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
-        ActiveRepairService.instance.registerParentRepairSession(parentRepairSession, FBUtilities.getBroadcastAddress(),
+        ActiveRepairService.instance.registerParentRepairSession(parentRepairSession, FBUtilities.getBroadcastAddressAndPort(),
                                                                  Arrays.asList(cfs), Arrays.asList(range), false,
                                                                  ActiveRepairService.UNREPAIRED_SSTABLE, false,
                                                                  PreviewKind.NONE);
@@ -128,8 +128,8 @@ public class LocalSyncTaskTest extends AbstractRepairTest
 
         // difference the trees
         // note: we reuse the same endpoint which is bogus in theory but fine here
-        TreeResponse r1 = new TreeResponse(InetAddress.getByName("127.0.0.1"), tree1);
-        TreeResponse r2 = new TreeResponse(InetAddress.getByName("127.0.0.2"), tree2);
+        TreeResponse r1 = new TreeResponse(InetAddressAndPort.getByName("127.0.0.1"), tree1);
+        TreeResponse r2 = new TreeResponse(InetAddressAndPort.getByName("127.0.0.2"), tree2);
         LocalSyncTask task = new LocalSyncTask(desc, r1, r2, NO_PENDING_REPAIR, false, PreviewKind.NONE);
         task.run();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java b/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java
index db76f73..2044106 100644
--- a/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java
+++ b/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.RepairRunnable.CommonRange;
 
 import static org.apache.cassandra.repair.RepairRunnable.filterCommonRanges;
@@ -53,7 +54,7 @@ public class RepairRunnableTest extends AbstractRepairTest
     {
         CommonRange cr1 = new CommonRange(Sets.newHashSet(PARTICIPANT1, PARTICIPANT2), Sets.newHashSet(RANGE1, RANGE2));
         CommonRange cr2 = new CommonRange(Sets.newHashSet(PARTICIPANT1, PARTICIPANT2, PARTICIPANT3), Sets.newHashSet(RANGE3));
-        Set<InetAddress> liveEndpoints = Sets.newHashSet(PARTICIPANT2, PARTICIPANT3); // PARTICIPANT1 is excluded
+        Set<InetAddressAndPort> liveEndpoints = Sets.newHashSet(PARTICIPANT2, PARTICIPANT3); // PARTICIPANT1 is excluded
 
         List<CommonRange> initial = Lists.newArrayList(cr1, cr2);
         List<CommonRange> expected = Lists.newArrayList(new CommonRange(Sets.newHashSet(PARTICIPANT2), Sets.newHashSet(RANGE1, RANGE2)),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
index 984218d..54f0511 100644
--- a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
+++ b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.repair;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.Arrays;
 import java.util.Set;
 import java.util.UUID;
@@ -35,7 +34,7 @@ import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
@@ -54,7 +53,7 @@ public class RepairSessionTest
     @Test
     public void testConviction() throws Exception
     {
-        InetAddress remote = InetAddress.getByName("127.0.0.2");
+        InetAddressAndPort remote = InetAddressAndPort.getByName("127.0.0.2");
         Gossiper.instance.initializeNodeUnsafe(remote, UUID.randomUUID(), 1);
 
         // Set up RepairSession
@@ -62,7 +61,7 @@ public class RepairSessionTest
         UUID sessionId = UUID.randomUUID();
         IPartitioner p = Murmur3Partitioner.instance;
         Range<Token> repairRange = new Range<>(p.getToken(ByteBufferUtil.bytes(0)), p.getToken(ByteBufferUtil.bytes(100)));
-        Set<InetAddress> endpoints = Sets.newHashSet(remote);
+        Set<InetAddressAndPort> endpoints = Sets.newHashSet(remote);
         RepairSession session = new RepairSession(parentSessionId, sessionId, Arrays.asList(repairRange),
                                                   "Keyspace1", RepairParallelism.SEQUENTIAL,
                                                   endpoints, false, false, false,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/ValidatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
index b45edc1..322772a 100644
--- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java
+++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
@@ -34,6 +33,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -95,7 +95,7 @@ public class ValidatorTest
 
         final CompletableFuture<MessageOut> outgoingMessageSink = registerOutgoingMessageSink();
 
-        InetAddress remote = InetAddress.getByName("127.0.0.2");
+        InetAddressAndPort remote = InetAddressAndPort.getByName("127.0.0.2");
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily);
 
@@ -134,7 +134,7 @@ public class ValidatorTest
 
         final CompletableFuture<MessageOut> outgoingMessageSink = registerOutgoingMessageSink();
 
-        InetAddress remote = InetAddress.getByName("127.0.0.2");
+        InetAddressAndPort remote = InetAddressAndPort.getByName("127.0.0.2");
 
         Validator validator = new Validator(desc, remote, 0, PreviewKind.NONE);
         validator.fail();
@@ -189,12 +189,12 @@ public class ValidatorTest
                                                cfs.getTableName(), Collections.singletonList(new Range<>(sstable.first.getToken(),
                                                                                                                 sstable.last.getToken())));
 
-        ActiveRepairService.instance.registerParentRepairSession(repairSessionId, FBUtilities.getBroadcastAddress(),
+        ActiveRepairService.instance.registerParentRepairSession(repairSessionId, FBUtilities.getBroadcastAddressAndPort(),
                                                                  Collections.singletonList(cfs), desc.ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE,
                                                                  false, PreviewKind.NONE);
 
         final CompletableFuture<MessageOut> outgoingMessageSink = registerOutgoingMessageSink();
-        Validator validator = new Validator(desc, FBUtilities.getBroadcastAddress(), 0, true, false, PreviewKind.NONE);
+        Validator validator = new Validator(desc, FBUtilities.getBroadcastAddressAndPort(), 0, true, false, PreviewKind.NONE);
         CompactionManager.instance.submitValidation(cfs, validator);
 
         MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS);
@@ -218,7 +218,7 @@ public class ValidatorTest
         final CompletableFuture<MessageOut> future = new CompletableFuture<>();
         MessagingService.instance().addMessageSink(new IMessageSink()
         {
-            public boolean allowOutgoingMessage(MessageOut message, int id, InetAddress to)
+            public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to)
             {
                 future.complete(message);
                 return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java b/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java
index 52a43e6..9693010 100644
--- a/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java
+++ b/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.asymmetric;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Iterator;
 
@@ -30,6 +29,7 @@ import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.TreeResponse;
 import org.apache.cassandra.utils.MerkleTree;
 import org.apache.cassandra.utils.MerkleTrees;
@@ -43,8 +43,8 @@ public class DifferenceHolderTest
     @Test
     public void testFromEmptyMerkleTrees() throws UnknownHostException
     {
-        InetAddress a1 = InetAddress.getByName("127.0.0.1");
-        InetAddress a2 = InetAddress.getByName("127.0.0.2");
+        InetAddressAndPort a1 = InetAddressAndPort.getByName("127.0.0.1");
+        InetAddressAndPort a2 = InetAddressAndPort.getByName("127.0.0.2");
 
         MerkleTrees mt1 = new MerkleTrees(Murmur3Partitioner.instance);
         MerkleTrees mt2 = new MerkleTrees(Murmur3Partitioner.instance);
@@ -64,8 +64,8 @@ public class DifferenceHolderTest
         IPartitioner partitioner = Murmur3Partitioner.instance;
         Range<Token> fullRange = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken());
         int maxsize = 16;
-        InetAddress a1 = InetAddress.getByName("127.0.0.1");
-        InetAddress a2 = InetAddress.getByName("127.0.0.2");
+        InetAddressAndPort a1 = InetAddressAndPort.getByName("127.0.0.1");
+        InetAddressAndPort a2 = InetAddressAndPort.getByName("127.0.0.2");
         // merkle tree building stolen from MerkleTreesTest:
         MerkleTrees mt1 = new MerkleTrees(partitioner);
         MerkleTrees mt2 = new MerkleTrees(partitioner);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/asymmetric/ReduceHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/asymmetric/ReduceHelperTest.java b/test/unit/org/apache/cassandra/repair/asymmetric/ReduceHelperTest.java
index 19c42fb..6c64b1a 100644
--- a/test/unit/org/apache/cassandra/repair/asymmetric/ReduceHelperTest.java
+++ b/test/unit/org/apache/cassandra/repair/asymmetric/ReduceHelperTest.java
@@ -19,11 +19,9 @@
 package org.apache.cassandra.repair.asymmetric;
 
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -33,13 +31,13 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 import org.junit.Test;
 
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static junit.framework.TestCase.fail;
 import static org.junit.Assert.assertEquals;
@@ -47,24 +45,24 @@ import static org.junit.Assert.assertTrue;
 
 public class ReduceHelperTest
 {
-    private static final InetAddress[] addresses;
-    private static final InetAddress A;
-    private static final InetAddress B;
-    private static final InetAddress C;
-    private static final InetAddress D;
-    private static final InetAddress E;
+    private static final InetAddressAndPort[] addresses;
+    private static final InetAddressAndPort A;
+    private static final InetAddressAndPort B;
+    private static final InetAddressAndPort C;
+    private static final InetAddressAndPort D;
+    private static final InetAddressAndPort E;
 
     static
     {
         try
         {
-            A = InetAddress.getByName("127.0.0.0");
-            B = InetAddress.getByName("127.0.0.1");
-            C = InetAddress.getByName("127.0.0.2");
-            D = InetAddress.getByName("127.0.0.3");
-            E = InetAddress.getByName("127.0.0.4");
+            A = InetAddressAndPort.getByName("127.0.0.0");
+            B = InetAddressAndPort.getByName("127.0.0.1");
+            C = InetAddressAndPort.getByName("127.0.0.2");
+            D = InetAddressAndPort.getByName("127.0.0.3");
+            E = InetAddressAndPort.getByName("127.0.0.4");
             // for diff creation in loops:
-            addresses = new InetAddress[]{ A, B, C, D, E };
+            addresses = new InetAddressAndPort[]{ A, B, C, D, E };
         }
         catch (UnknownHostException e)
         {
@@ -89,7 +87,7 @@ public class ReduceHelperTest
         C             x   x
         D                 =
          */
-        Map<InetAddress, HostDifferences> differences = new HashMap<>();
+        Map<InetAddressAndPort, HostDifferences> differences = new HashMap<>();
         for (int i = 0; i < 4; i++)
         {
             HostDifferences hostDiffs = new HostDifferences();
@@ -105,7 +103,7 @@ public class ReduceHelperTest
 
         }
         DifferenceHolder differenceHolder = new DifferenceHolder(differences);
-        Map<InetAddress, IncomingRepairStreamTracker> tracker = ReduceHelper.createIncomingRepairStreamTrackers(differenceHolder);
+        Map<InetAddressAndPort, IncomingRepairStreamTracker> tracker = ReduceHelper.createIncomingRepairStreamTrackers(differenceHolder);
 
         assertEquals(set(set(C), set(E,D)), streams(tracker.get(A)));
         assertEquals(set(set(C), set(E,D)), streams(tracker.get(B)));
@@ -113,7 +111,7 @@ public class ReduceHelperTest
         assertEquals(set(set(A,B), set(C)), streams(tracker.get(D)));
         assertEquals(set(set(A,B), set(C)), streams(tracker.get(E)));
 
-        ImmutableMap<InetAddress, HostDifferences> reduced = ReduceHelper.reduce(differenceHolder, (x,y) -> y);
+        ImmutableMap<InetAddressAndPort, HostDifferences> reduced = ReduceHelper.reduce(differenceHolder, (x, y) -> y);
 
         HostDifferences n0 = reduced.get(A);
         assertEquals(0, n0.get(A).size());
@@ -163,7 +161,7 @@ public class ReduceHelperTest
         C             x   x
         D                 =
          */
-        Map<InetAddress, HostDifferences> differences = new HashMap<>();
+        Map<InetAddressAndPort, HostDifferences> differences = new HashMap<>();
         for (int i = 0; i < 4; i++)
         {
             HostDifferences hostDifferences = new HostDifferences();
@@ -179,7 +177,7 @@ public class ReduceHelperTest
         }
 
         DifferenceHolder differenceHolder = new DifferenceHolder(differences);
-        Map<InetAddress, IncomingRepairStreamTracker> tracker = ReduceHelper.createIncomingRepairStreamTrackers(differenceHolder);
+        Map<InetAddressAndPort, IncomingRepairStreamTracker> tracker = ReduceHelper.createIncomingRepairStreamTrackers(differenceHolder);
         assertEquals(set(set(C), set(E, D)), streams(tracker.get(A)));
         assertEquals(set(set(C), set(E, D)), streams(tracker.get(B)));
         assertEquals(set(set(A, B), set(E, D)), streams(tracker.get(C)));
@@ -187,7 +185,7 @@ public class ReduceHelperTest
         assertEquals(set(set(A, B), set(C)), streams(tracker.get(E)));
 
         // if there is an option, never stream from node 1:
-        ImmutableMap<InetAddress, HostDifferences> reduced = ReduceHelper.reduce(differenceHolder, (x,y) -> Sets.difference(y, set(B)));
+        ImmutableMap<InetAddressAndPort, HostDifferences> reduced = ReduceHelper.reduce(differenceHolder, (x,y) -> Sets.difference(y, set(B)));
 
         HostDifferences n0 = reduced.get(A);
         assertEquals(0, n0.get(A).size());
@@ -223,7 +221,7 @@ public class ReduceHelperTest
         assertEquals(0, n4.get(E).size());
     }
 
-    private Iterable<Set<InetAddress>> streams(IncomingRepairStreamTracker incomingRepairStreamTracker)
+    private Iterable<Set<InetAddressAndPort>> streams(IncomingRepairStreamTracker incomingRepairStreamTracker)
     {
         return incomingRepairStreamTracker.getIncoming().values().iterator().next().allStreams();
     }
@@ -248,12 +246,12 @@ public class ReduceHelperTest
          B streams (0, 50] from {C}, (50, 100] from {A, C}
          C streams (0, 50] from {A, B}, (50, 100] from B
          */
-        Map<InetAddress, HostDifferences> differences = new HashMap<>();
+        Map<InetAddressAndPort, HostDifferences> differences = new HashMap<>();
         addDifference(A, differences, B, list(range(50, 100)));
         addDifference(A, differences, C, list(range(0, 50)));
         addDifference(B, differences, C, list(range(0, 100)));
         DifferenceHolder differenceHolder = new DifferenceHolder(differences);
-        Map<InetAddress, IncomingRepairStreamTracker> tracker = ReduceHelper.createIncomingRepairStreamTrackers(differenceHolder);
+        Map<InetAddressAndPort, IncomingRepairStreamTracker> tracker = ReduceHelper.createIncomingRepairStreamTrackers(differenceHolder);
         assertEquals(set(set(C)), tracker.get(A).getIncoming().get(range(0, 50)).allStreams());
         assertEquals(set(set(B)), tracker.get(A).getIncoming().get(range(50, 100)).allStreams());
         assertEquals(set(set(C)), tracker.get(B).getIncoming().get(range(0, 50)).allStreams());
@@ -261,7 +259,7 @@ public class ReduceHelperTest
         assertEquals(set(set(A,B)), tracker.get(C).getIncoming().get(range(0, 50)).allStreams());
         assertEquals(set(set(B)), tracker.get(C).getIncoming().get(range(50, 100)).allStreams());
 
-        ImmutableMap<InetAddress, HostDifferences> reduced = ReduceHelper.reduce(differenceHolder, (x, y) -> y);
+        ImmutableMap<InetAddressAndPort, HostDifferences> reduced = ReduceHelper.reduce(differenceHolder, (x, y) -> y);
 
         HostDifferences n0 = reduced.get(A);
 
@@ -270,7 +268,7 @@ public class ReduceHelperTest
 
         HostDifferences n1 = reduced.get(B);
         assertEquals(0, n1.get(B).size());
-        if (n1.get(A) != null)
+        if (!n1.get(A).isEmpty())
         {
             assertTrue(n1.get(C).equals(list(range(0, 50))));
             assertTrue(n1.get(A).equals(list(range(50, 100))));
@@ -281,7 +279,7 @@ public class ReduceHelperTest
         }
         HostDifferences n2 = reduced.get(C);
         assertEquals(0, n2.get(C).size());
-        if (n2.get(A) != null)
+        if (!n2.get(A).isEmpty())
         {
             assertTrue(n2.get(A).equals(list(range(0,50))));
             assertTrue(n2.get(B).equals(list(range(50, 100))));
@@ -312,13 +310,13 @@ public class ReduceHelperTest
          B == C on (5, 10], (40, 45]
          */
 
-        Map<InetAddress, HostDifferences> differences = new HashMap<>();
+        Map<InetAddressAndPort, HostDifferences> differences = new HashMap<>();
         addDifference(A, differences, B, list(range(5, 45)));
         addDifference(A, differences, C, list(range(0, 10), range(40,50)));
         addDifference(B, differences, C, list(range(0, 5), range(10,40), range(45,50)));
 
         DifferenceHolder differenceHolder = new DifferenceHolder(differences);
-        Map<InetAddress, IncomingRepairStreamTracker> tracker = ReduceHelper.createIncomingRepairStreamTrackers(differenceHolder);
+        Map<InetAddressAndPort, IncomingRepairStreamTracker> tracker = ReduceHelper.createIncomingRepairStreamTrackers(differenceHolder);
 
         Map<Range<Token>, StreamFromOptions> ranges = tracker.get(A).getIncoming();
         assertEquals(5, ranges.size());
@@ -344,21 +342,21 @@ public class ReduceHelperTest
         assertEquals(set(set(B)), ranges.get(range(10, 40)).allStreams());
         assertEquals(set(set(A)), ranges.get(range(40, 45)).allStreams());
         assertEquals(set(set(A,B)), ranges.get(range(45, 50)).allStreams());
-        ImmutableMap<InetAddress, HostDifferences> reduced = ReduceHelper.reduce(differenceHolder, (x, y) -> y);
+        ImmutableMap<InetAddressAndPort, HostDifferences> reduced = ReduceHelper.reduce(differenceHolder, (x, y) -> y);
 
         assertNoOverlap(A, reduced.get(A), list(range(0, 50)));
         assertNoOverlap(B, reduced.get(B), list(range(0, 50)));
         assertNoOverlap(C, reduced.get(C), list(range(0, 50)));
     }
 
-    private void assertNoOverlap(InetAddress incomingNode, HostDifferences node, List<Range<Token>> expectedAfterNormalize)
+    private void assertNoOverlap(InetAddressAndPort incomingNode, HostDifferences node, List<Range<Token>> expectedAfterNormalize)
     {
         Set<Range<Token>> allRanges = new HashSet<>();
-        Set<InetAddress> remoteNodes = Sets.newHashSet(A,B,C);
+        Set<InetAddressAndPort> remoteNodes = Sets.newHashSet(A,B,C);
         remoteNodes.remove(incomingNode);
-        Iterator<InetAddress> iter = remoteNodes.iterator();
+        Iterator<InetAddressAndPort> iter = remoteNodes.iterator();
         allRanges.addAll(node.get(iter.next()));
-        InetAddress i = iter.next();
+        InetAddressAndPort i = iter.next();
         for (Range<Token> r : node.get(i))
         {
             for (Range<Token> existing : allRanges)
@@ -379,14 +377,14 @@ public class ReduceHelperTest
         return ranges;
     }
 
-    private static Set<InetAddress> set(InetAddress ... elem)
+    private static Set<InetAddressAndPort> set(InetAddressAndPort ... elem)
     {
         return Sets.newHashSet(elem);
     }
     @SafeVarargs
-    private static Set<Set<InetAddress>> set(Set<InetAddress> ... elem)
+    private static Set<Set<InetAddressAndPort>> set(Set<InetAddressAndPort> ... elem)
     {
-        Set<Set<InetAddress>> ret = Sets.newHashSet();
+        Set<Set<InetAddressAndPort>> ret = Sets.newHashSet();
         ret.addAll(Arrays.asList(elem));
         return ret;
     }
@@ -418,7 +416,7 @@ public class ReduceHelperTest
         assertTrue(r1.size() > 0 ^ r2.size() > 0);
     }
 
-    private void addDifference(InetAddress host1, Map<InetAddress, HostDifferences> differences, InetAddress host2, List<Range<Token>> ranges)
+    private void addDifference(InetAddressAndPort host1, Map<InetAddressAndPort, HostDifferences> differences, InetAddressAndPort host2, List<Range<Token>> ranges)
     {
         differences.computeIfAbsent(host1, (x) -> new HostDifferences()).add(host2, ranges);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/asymmetric/StreamFromOptionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/asymmetric/StreamFromOptionsTest.java b/test/unit/org/apache/cassandra/repair/asymmetric/StreamFromOptionsTest.java
index 3ba3cfe..e2a7700 100644
--- a/test/unit/org/apache/cassandra/repair/asymmetric/StreamFromOptionsTest.java
+++ b/test/unit/org/apache/cassandra/repair/asymmetric/StreamFromOptionsTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.asymmetric;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collections;
 import java.util.HashSet;
@@ -30,6 +29,7 @@ import org.junit.Test;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static junit.framework.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -41,16 +41,16 @@ public class StreamFromOptionsTest
     public void addAllDiffingTest() throws UnknownHostException
     {
         StreamFromOptions sfo = new StreamFromOptions(new MockDiffs(true), range(0, 10));
-        Set<InetAddress> toAdd = new HashSet<>();
-        toAdd.add(InetAddress.getByName("127.0.0.1"));
-        toAdd.add(InetAddress.getByName("127.0.0.2"));
-        toAdd.add(InetAddress.getByName("127.0.0.3"));
+        Set<InetAddressAndPort> toAdd = new HashSet<>();
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.1"));
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.2"));
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.3"));
         toAdd.forEach(sfo::add);
 
         // if all added have differences, each set will contain a single host
         assertEquals(3, Iterables.size(sfo.allStreams()));
-        Set<InetAddress> allStreams = new HashSet<>();
-        for (Set<InetAddress> streams : sfo.allStreams())
+        Set<InetAddressAndPort> allStreams = new HashSet<>();
+        for (Set<InetAddressAndPort> streams : sfo.allStreams())
         {
             assertEquals(1, streams.size());
             allStreams.addAll(streams);
@@ -62,10 +62,10 @@ public class StreamFromOptionsTest
     public void addAllMatchingTest() throws UnknownHostException
     {
         StreamFromOptions sfo = new StreamFromOptions(new MockDiffs(false), range(0, 10));
-        Set<InetAddress> toAdd = new HashSet<>();
-        toAdd.add(InetAddress.getByName("127.0.0.1"));
-        toAdd.add(InetAddress.getByName("127.0.0.2"));
-        toAdd.add(InetAddress.getByName("127.0.0.3"));
+        Set<InetAddressAndPort> toAdd = new HashSet<>();
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.1"));
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.2"));
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.3"));
         toAdd.forEach(sfo::add);
 
         // if all added match, the set will contain all hosts
@@ -83,10 +83,10 @@ public class StreamFromOptionsTest
     private void splitTestHelper(boolean diffing) throws UnknownHostException
     {
         StreamFromOptions sfo = new StreamFromOptions(new MockDiffs(diffing), range(0, 10));
-        Set<InetAddress> toAdd = new HashSet<>();
-        toAdd.add(InetAddress.getByName("127.0.0.1"));
-        toAdd.add(InetAddress.getByName("127.0.0.2"));
-        toAdd.add(InetAddress.getByName("127.0.0.3"));
+        Set<InetAddressAndPort> toAdd = new HashSet<>();
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.1"));
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.2"));
+        toAdd.add(InetAddressAndPort.getByName("127.0.0.3"));
         toAdd.forEach(sfo::add);
         StreamFromOptions sfo1 = sfo.copy(range(0, 5));
         StreamFromOptions sfo2 = sfo.copy(range(5, 10));
@@ -95,8 +95,8 @@ public class StreamFromOptionsTest
         assertEquals(range(5, 10), sfo2.range);
         assertTrue(Iterables.elementsEqual(sfo1.allStreams(), sfo2.allStreams()));
         // verify the backing set is not shared between the copies:
-        sfo1.add(InetAddress.getByName("127.0.0.4"));
-        sfo2.add(InetAddress.getByName("127.0.0.5"));
+        sfo1.add(InetAddressAndPort.getByName("127.0.0.4"));
+        sfo2.add(InetAddressAndPort.getByName("127.0.0.5"));
         assertFalse(Iterables.elementsEqual(sfo1.allStreams(), sfo2.allStreams()));
     }
 
@@ -116,7 +116,7 @@ public class StreamFromOptionsTest
         }
 
         @Override
-        public boolean hasDifferenceBetween(InetAddress node1, InetAddress node2, Range<Token> range)
+        public boolean hasDifferenceBetween(InetAddressAndPort node1, InetAddressAndPort node2, Range<Token> range)
         {
             return hasDifference;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/consistent/AbstractConsistentSessionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/AbstractConsistentSessionTest.java b/test/unit/org/apache/cassandra/repair/consistent/AbstractConsistentSessionTest.java
index 367fea9..4570328 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/AbstractConsistentSessionTest.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/AbstractConsistentSessionTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Set;
 import java.util.UUID;
@@ -33,6 +32,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -41,19 +41,19 @@ import org.apache.cassandra.utils.UUIDGen;
 @Ignore
 public abstract class AbstractConsistentSessionTest
 {
-    protected static final InetAddress COORDINATOR;
-    protected static final InetAddress PARTICIPANT1;
-    protected static final InetAddress PARTICIPANT2;
-    protected static final InetAddress PARTICIPANT3;
+    protected static final InetAddressAndPort COORDINATOR;
+    protected static final InetAddressAndPort PARTICIPANT1;
+    protected static final InetAddressAndPort PARTICIPANT2;
+    protected static final InetAddressAndPort PARTICIPANT3;
 
     static
     {
         try
         {
-            COORDINATOR = InetAddress.getByName("10.0.0.1");
-            PARTICIPANT1 = InetAddress.getByName("10.0.0.1");
-            PARTICIPANT2 = InetAddress.getByName("10.0.0.2");
-            PARTICIPANT3 = InetAddress.getByName("10.0.0.3");
+            COORDINATOR = InetAddressAndPort.getByName("10.0.0.1");
+            PARTICIPANT1 = InetAddressAndPort.getByName("10.0.0.1");
+            PARTICIPANT2 = InetAddressAndPort.getByName("10.0.0.2");
+            PARTICIPANT3 = InetAddressAndPort.getByName("10.0.0.3");
         }
         catch (UnknownHostException e)
         {
@@ -64,7 +64,7 @@ public abstract class AbstractConsistentSessionTest
         DatabaseDescriptor.daemonInitialization();
     }
 
-    protected static final Set<InetAddress> PARTICIPANTS = ImmutableSet.of(PARTICIPANT1, PARTICIPANT2, PARTICIPANT3);
+    protected static final Set<InetAddressAndPort> PARTICIPANTS = ImmutableSet.of(PARTICIPANT1, PARTICIPANT2, PARTICIPANT3);
 
     protected static Token t(int v)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java
index fb312c3..5d054d3 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -35,6 +34,7 @@ import com.google.common.util.concurrent.SettableFuture;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.AbstractRepairTest;
 import org.apache.cassandra.repair.RepairSessionResult;
 import org.apache.cassandra.repair.messages.FailSession;
@@ -77,7 +77,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         return new RepairSessionResult(coordinator.sessionID, "ks", coordinator.ranges, null, false);
     }
 
-    private static void assertMessageSent(InstrumentedCoordinatorSession coordinator, InetAddress participant, RepairMessage expected)
+    private static void assertMessageSent(InstrumentedCoordinatorSession coordinator, InetAddressAndPort participant, RepairMessage expected)
     {
         Assert.assertTrue(coordinator.sentMessages.containsKey(participant));
         Assert.assertEquals(1, coordinator.sentMessages.get(participant).size());
@@ -91,9 +91,9 @@ public class CoordinatorSessionTest extends AbstractRepairTest
             super(builder);
         }
 
-        Map<InetAddress, List<RepairMessage>> sentMessages = new HashMap<>();
+        Map<InetAddressAndPort, List<RepairMessage>> sentMessages = new HashMap<>();
 
-        protected void sendMessage(InetAddress destination, RepairMessage message)
+        protected void sendMessage(InetAddressAndPort destination, RepairMessage message)
         {
             if (!sentMessages.containsKey(destination))
             {
@@ -189,7 +189,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
 
         coordinator.fail();
         Assert.assertEquals(FAILED, coordinator.getState());
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             assertMessageSent(coordinator, participant, new FailSession(coordinator.sessionID));
         }
@@ -221,7 +221,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         Assert.assertTrue(coordinator.sentMessages.isEmpty());
         ListenableFuture sessionResult = coordinator.execute(sessionSupplier, hasFailures);
 
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
 
             RepairMessage expected = new PrepareConsistentRequest(coordinator.sessionID, COORDINATOR, new HashSet<>(PARTICIPANTS));
@@ -254,7 +254,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         repairFuture.set(results);
 
         // propose messages should have been sent once all repair sessions completed successfully
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             RepairMessage expected = new FinalizePropose(coordinator.sessionID);
             assertMessageSent(coordinator, participant, expected);
@@ -277,7 +277,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         Assert.assertTrue(coordinator.finalizeCommitCalled);
 
         Assert.assertEquals(ConsistentSession.State.FINALIZED, coordinator.getState());
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             RepairMessage expected = new FinalizeCommit(coordinator.sessionID);
             assertMessageSent(coordinator, participant, expected);
@@ -304,7 +304,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         Assert.assertFalse(repairSubmitted.get());
         Assert.assertTrue(coordinator.sentMessages.isEmpty());
         ListenableFuture sessionResult = coordinator.execute(sessionSupplier, hasFailures);
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             PrepareConsistentRequest expected = new PrepareConsistentRequest(coordinator.sessionID, COORDINATOR, new HashSet<>(PARTICIPANTS));
             assertMessageSent(coordinator, participant, expected);
@@ -339,7 +339,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         Assert.assertTrue(coordinator.failCalled);
 
         // all participants should have been notified of session failure
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             RepairMessage expected = new FailSession(coordinator.sessionID);
             assertMessageSent(coordinator, participant, expected);
@@ -366,7 +366,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         Assert.assertFalse(repairSubmitted.get());
         Assert.assertTrue(coordinator.sentMessages.isEmpty());
         ListenableFuture sessionResult = coordinator.execute(sessionSupplier, hasFailures);
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             PrepareConsistentRequest expected = new PrepareConsistentRequest(coordinator.sessionID, COORDINATOR, new HashSet<>(PARTICIPANTS));
             assertMessageSent(coordinator, participant, expected);
@@ -394,7 +394,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         Assert.assertFalse(repairSubmitted.get());
 
         // all participants should have been notified of session failure
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             RepairMessage expected = new FailSession(coordinator.sessionID);
             assertMessageSent(coordinator, participant, expected);
@@ -422,7 +422,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         Assert.assertTrue(coordinator.sentMessages.isEmpty());
         ListenableFuture sessionResult = coordinator.execute(sessionSupplier, hasFailures);
 
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
 
             RepairMessage expected = new PrepareConsistentRequest(coordinator.sessionID, COORDINATOR, new HashSet<>(PARTICIPANTS));
@@ -455,7 +455,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         repairFuture.set(results);
 
         // propose messages should have been sent once all repair sessions completed successfully
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             RepairMessage expected = new FinalizePropose(coordinator.sessionID);
             assertMessageSent(coordinator, participant, expected);
@@ -481,7 +481,7 @@ public class CoordinatorSessionTest extends AbstractRepairTest
         Assert.assertEquals(ConsistentSession.State.FAILED, coordinator.getState());
 
         // failure messages should have been sent to all participants
-        for (InetAddress participant : PARTICIPANTS)
+        for (InetAddressAndPort participant : PARTICIPANTS)
         {
             RepairMessage expected = new FailSession(coordinator.sessionID);
             assertMessageSent(coordinator, participant, expected);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java
index b40e185..9bf4270 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.Set;
 import java.util.UUID;
 
@@ -29,6 +28,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.AbstractRepairTest;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
@@ -55,9 +55,9 @@ public class CoordinatorSessionsTest extends AbstractRepairTest
         }
 
         int prepareResponseCalls = 0;
-        InetAddress preparePeer = null;
+        InetAddressAndPort preparePeer = null;
         boolean prepareSuccess = false;
-        public synchronized void handlePrepareResponse(InetAddress participant, boolean success)
+        public synchronized void handlePrepareResponse(InetAddressAndPort participant, boolean success)
         {
             prepareResponseCalls++;
             preparePeer = participant;
@@ -65,9 +65,9 @@ public class CoordinatorSessionsTest extends AbstractRepairTest
         }
 
         int finalizePromiseCalls = 0;
-        InetAddress promisePeer = null;
+        InetAddressAndPort promisePeer = null;
         boolean promiseSuccess = false;
-        public synchronized void handleFinalizePromise(InetAddress participant, boolean success)
+        public synchronized void handleFinalizePromise(InetAddressAndPort participant, boolean success)
         {
             finalizePromiseCalls++;
             promisePeer = participant;
@@ -93,7 +93,7 @@ public class CoordinatorSessionsTest extends AbstractRepairTest
             return (InstrumentedCoordinatorSession) super.getSession(sessionId);
         }
 
-        public InstrumentedCoordinatorSession registerSession(UUID sessionId, Set<InetAddress> peers)
+        public InstrumentedCoordinatorSession registerSession(UUID sessionId, Set<InetAddressAndPort> peers)
         {
             return (InstrumentedCoordinatorSession) super.registerSession(sessionId, peers);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java
index 6808efe..3ea888d 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java
@@ -18,10 +18,10 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.Set;
 import java.util.UUID;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.ActiveRepairService;
 
 /**
@@ -36,7 +36,7 @@ public class LocalSessionAccessor
         ARS.consistent.local.start();
     }
 
-    public static void prepareUnsafe(UUID sessionID, InetAddress coordinator, Set<InetAddress> peers)
+    public static void prepareUnsafe(UUID sessionID, InetAddressAndPort coordinator, Set<InetAddressAndPort> peers)
     {
         ActiveRepairService.ParentRepairSession prs = ARS.getParentRepairSession(sessionID);
         assert prs != null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
index 6e6d222..5fa43a9 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -42,6 +41,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.repair.AbstractRepairTest;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.SchemaConstants;
@@ -106,20 +106,20 @@ public class LocalSessionTest extends AbstractRepairTest
         }
     }
 
-    private static void assertNoMessagesSent(InstrumentedLocalSessions sessions, InetAddress to)
+    private static void assertNoMessagesSent(InstrumentedLocalSessions sessions, InetAddressAndPort to)
     {
         Assert.assertNull(sessions.sentMessages.get(to));
     }
 
-    private static void assertMessagesSent(InstrumentedLocalSessions sessions, InetAddress to, RepairMessage... expected)
+    private static void assertMessagesSent(InstrumentedLocalSessions sessions, InetAddressAndPort to, RepairMessage... expected)
     {
         Assert.assertEquals(Lists.newArrayList(expected), sessions.sentMessages.get(to));
     }
 
     static class InstrumentedLocalSessions extends LocalSessions
     {
-        Map<InetAddress, List<RepairMessage>> sentMessages = new HashMap<>();
-        protected void sendMessage(InetAddress destination, RepairMessage message)
+        Map<InetAddressAndPort, List<RepairMessage>> sentMessages = new HashMap<>();
+        protected void sendMessage(InetAddressAndPort destination, RepairMessage message)
         {
             if (!sentMessages.containsKey(destination))
             {
@@ -159,12 +159,13 @@ public class LocalSessionTest extends AbstractRepairTest
             return getSession(sessionID);
         }
 
-        protected InetAddress getBroadcastAddress()
+        @Override
+        protected InetAddressAndPort getBroadcastAddressAndPort()
         {
             return PARTICIPANT1;
         }
 
-        protected boolean isAlive(InetAddress address)
+        protected boolean isAlive(InetAddressAndPort address)
         {
             return true;
         }
@@ -811,7 +812,7 @@ public class LocalSessionTest extends AbstractRepairTest
         sessions.start();
         Assert.assertNotNull(sessions.getSession(session.sessionID));
 
-        QueryProcessor.instance.executeInternal("DELETE participants FROM system.repairs WHERE parent_id=?", session.sessionID);
+        QueryProcessor.instance.executeInternal("DELETE participants, participants_wp FROM system.repairs WHERE parent_id=?", session.sessionID);
 
         sessions = new LocalSessions();
         sessions.start();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/repair/consistent/PendingAntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/PendingAntiCompactionTest.java b/test/unit/org/apache/cassandra/repair/consistent/PendingAntiCompactionTest.java
index 5aeab3e..213cdd3 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/PendingAntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/PendingAntiCompactionTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -43,6 +42,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
@@ -148,7 +148,7 @@ public class PendingAntiCompactionTest
 
         // create a session so the anti compaction can fine it
         UUID sessionID = UUIDGen.getTimeUUID();
-        ActiveRepairService.instance.registerParentRepairSession(sessionID, InetAddress.getLocalHost(), Lists.newArrayList(cfs), ranges, true, 1, true, PreviewKind.NONE);
+        ActiveRepairService.instance.registerParentRepairSession(sessionID, InetAddressAndPort.getLocalHost(), Lists.newArrayList(cfs), ranges, true, 1, true, PreviewKind.NONE);
 
         PendingAntiCompaction pac;
         ExecutorService executor = Executors.newSingleThreadExecutor();
@@ -352,7 +352,7 @@ public class PendingAntiCompactionTest
         PendingAntiCompaction.AcquireResult result = acquisitionCallable.call();
         UUID sessionID = UUIDGen.getTimeUUID();
         ActiveRepairService.instance.registerParentRepairSession(sessionID,
-                                                                 InetAddress.getByName("127.0.0.1"),
+                                                                 InetAddressAndPort.getByName("127.0.0.1"),
                                                                  Lists.newArrayList(cfs),
                                                                  FULL_RANGE,
                                                                  true,0,


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


[11/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 0276238..c600789 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.service;
 
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 import java.util.concurrent.*;
@@ -61,6 +60,7 @@ import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
 import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
@@ -208,7 +208,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                                              Collection<Range<Token>> range,
                                              String keyspace,
                                              RepairParallelism parallelismDegree,
-                                             Set<InetAddress> endpoints,
+                                             Set<InetAddressAndPort> endpoints,
                                              boolean isIncremental,
                                              boolean pullRepair,
                                              boolean force,
@@ -297,12 +297,12 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
      *
      * @return neighbors with whom we share the provided range
      */
-    public static Set<InetAddress> getNeighbors(String keyspaceName, Collection<Range<Token>> keyspaceLocalRanges,
-                                                Range<Token> toRepair, Collection<String> dataCenters,
-                                                Collection<String> hosts)
+    public static Set<InetAddressAndPort> getNeighbors(String keyspaceName, Collection<Range<Token>> keyspaceLocalRanges,
+                                                       Range<Token> toRepair, Collection<String> dataCenters,
+                                                       Collection<String> hosts)
     {
         StorageService ss = StorageService.instance;
-        Map<Range<Token>, List<InetAddress>> replicaSets = ss.getRangeToAddressMap(keyspaceName);
+        Map<Range<Token>, List<InetAddressAndPort>> replicaSets = ss.getRangeToAddressMap(keyspaceName);
         Range<Token> rangeSuperSet = null;
         for (Range<Token> range : keyspaceLocalRanges)
         {
@@ -322,17 +322,17 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         if (rangeSuperSet == null || !replicaSets.containsKey(rangeSuperSet))
             return Collections.emptySet();
 
-        Set<InetAddress> neighbors = new HashSet<>(replicaSets.get(rangeSuperSet));
-        neighbors.remove(FBUtilities.getBroadcastAddress());
+        Set<InetAddressAndPort> neighbors = new HashSet<>(replicaSets.get(rangeSuperSet));
+        neighbors.remove(FBUtilities.getBroadcastAddressAndPort());
 
         if (dataCenters != null && !dataCenters.isEmpty())
         {
             TokenMetadata.Topology topology = ss.getTokenMetadata().cloneOnlyTokenMap().getTopology();
-            Set<InetAddress> dcEndpoints = Sets.newHashSet();
-            Multimap<String,InetAddress> dcEndpointsMap = topology.getDatacenterEndpoints();
+            Set<InetAddressAndPort> dcEndpoints = Sets.newHashSet();
+            Multimap<String,InetAddressAndPort> dcEndpointsMap = topology.getDatacenterEndpoints();
             for (String dc : dataCenters)
             {
-                Collection<InetAddress> c = dcEndpointsMap.get(dc);
+                Collection<InetAddressAndPort> c = dcEndpointsMap.get(dc);
                 if (c != null)
                    dcEndpoints.addAll(c);
             }
@@ -340,13 +340,13 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         }
         else if (hosts != null && !hosts.isEmpty())
         {
-            Set<InetAddress> specifiedHost = new HashSet<>();
+            Set<InetAddressAndPort> specifiedHost = new HashSet<>();
             for (final String host : hosts)
             {
                 try
                 {
-                    final InetAddress endpoint = InetAddress.getByName(host.trim());
-                    if (endpoint.equals(FBUtilities.getBroadcastAddress()) || neighbors.contains(endpoint))
+                    final InetAddressAndPort endpoint = InetAddressAndPort.getByName(host.trim());
+                    if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) || neighbors.contains(endpoint))
                         specifiedHost.add(endpoint);
                 }
                 catch (UnknownHostException e)
@@ -355,7 +355,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                 }
             }
 
-            if (!specifiedHost.contains(FBUtilities.getBroadcastAddress()))
+            if (!specifiedHost.contains(FBUtilities.getBroadcastAddressAndPort()))
                 throw new IllegalArgumentException("The current host must be part of the repair");
 
             if (specifiedHost.size() <= 1)
@@ -366,7 +366,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                 throw new IllegalArgumentException(String.format(msg, hosts, toRepair, neighbors));
             }
 
-            specifiedHost.remove(FBUtilities.getBroadcastAddress());
+            specifiedHost.remove(FBUtilities.getBroadcastAddressAndPort());
             return specifiedHost;
 
         }
@@ -393,7 +393,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         }
     }
 
-    public UUID prepareForRepair(UUID parentRepairSession, InetAddress coordinator, Set<InetAddress> endpoints, RepairOption options, List<ColumnFamilyStore> columnFamilyStores)
+    public UUID prepareForRepair(UUID parentRepairSession, InetAddressAndPort coordinator, Set<InetAddressAndPort> endpoints, RepairOption options, List<ColumnFamilyStore> columnFamilyStores)
     {
         long repairedAt = getRepairedAt(options);
         registerParentRepairSession(parentRepairSession, coordinator, columnFamilyStores, options.getRanges(), options.isIncremental(), repairedAt, options.isGlobal(), options.getPreviewKind());
@@ -412,10 +412,10 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                 return false;
             }
 
-            public void onFailure(InetAddress from, RequestFailureReason failureReason)
+            public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
             {
                 status.set(false);
-                failedNodes.add(from.getHostAddress());
+                failedNodes.add(from.toString());
                 prepareLatch.countDown();
             }
         };
@@ -424,7 +424,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         for (ColumnFamilyStore cfs : columnFamilyStores)
             tableIds.add(cfs.metadata.id);
 
-        for (InetAddress neighbour : endpoints)
+        for (InetAddressAndPort neighbour : endpoints)
         {
             if (FailureDetector.instance.isAlive(neighbour))
             {
@@ -471,7 +471,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         throw new RuntimeException(errorMsg);
     }
 
-    public synchronized void registerParentRepairSession(UUID parentRepairSession, InetAddress coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, boolean isIncremental, long repairedAt, boolean isGlobal, PreviewKind previewKind)
+    public synchronized void registerParentRepairSession(UUID parentRepairSession, InetAddressAndPort coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, boolean isIncremental, long repairedAt, boolean isGlobal, PreviewKind previewKind)
     {
         assert isIncremental || repairedAt == ActiveRepairService.UNREPAIRED_SSTABLE;
         if (!registeredForEndpointChanges)
@@ -517,7 +517,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         return parentRepairSessions.remove(parentSessionId);
     }
 
-    public void handleMessage(InetAddress endpoint, RepairMessage message)
+    public void handleMessage(InetAddressAndPort endpoint, RepairMessage message)
     {
         RepairJobDesc desc = message.desc;
         RepairSession session = sessions.get(desc.sessionId);
@@ -551,10 +551,10 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         public final boolean isIncremental;
         public final boolean isGlobal;
         public final long repairedAt;
-        public final InetAddress coordinator;
+        public final InetAddressAndPort coordinator;
         public final PreviewKind previewKind;
 
-        public ParentRepairSession(InetAddress coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, boolean isIncremental, long repairedAt, boolean isGlobal, PreviewKind previewKind)
+        public ParentRepairSession(InetAddressAndPort coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, boolean isIncremental, long repairedAt, boolean isGlobal, PreviewKind previewKind)
         {
             this.coordinator = coordinator;
             for (ColumnFamilyStore cfs : columnFamilyStores)
@@ -636,18 +636,18 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
     If the coordinator node dies we should remove the parent repair session from the other nodes.
     This uses the same notifications as we get in RepairSession
      */
-    public void onJoin(InetAddress endpoint, EndpointState epState) {}
-    public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
-    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) {}
-    public void onAlive(InetAddress endpoint, EndpointState state) {}
-    public void onDead(InetAddress endpoint, EndpointState state) {}
+    public void onJoin(InetAddressAndPort endpoint, EndpointState epState) {}
+    public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
+    public void onChange(InetAddressAndPort endpoint, ApplicationState state, VersionedValue value) {}
+    public void onAlive(InetAddressAndPort endpoint, EndpointState state) {}
+    public void onDead(InetAddressAndPort endpoint, EndpointState state) {}
 
-    public void onRemove(InetAddress endpoint)
+    public void onRemove(InetAddressAndPort endpoint)
     {
         convict(endpoint, Double.MAX_VALUE);
     }
 
-    public void onRestart(InetAddress endpoint, EndpointState state)
+    public void onRestart(InetAddressAndPort endpoint, EndpointState state)
     {
         convict(endpoint, Double.MAX_VALUE);
     }
@@ -661,7 +661,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
      * @param ep  endpoint to be convicted
      * @param phi the value of phi with with ep was convicted
      */
-    public void convict(InetAddress ep, double phi)
+    public void convict(InetAddressAndPort ep, double phi)
     {
         // We want a higher confidence in the failure detection than usual because failing a repair wrongly has a high cost.
         if (phi < 2 * DatabaseDescriptor.getPhiConvictThreshold() || parentRepairSessions.isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java b/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java
index 4e26101..e373fb6 100644
--- a/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java
@@ -18,12 +18,12 @@
 
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
 import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.exceptions.WriteFailureException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageIn;
 
 public class BatchlogResponseHandler<T> extends AbstractWriteResponseHandler<T>
@@ -59,7 +59,7 @@ public class BatchlogResponseHandler<T> extends AbstractWriteResponseHandler<T>
         return wrapped.isLatencyForSnitch();
     }
 
-    public void onFailure(InetAddress from, RequestFailureReason failureReason)
+    public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
     {
         wrapped.onFailure(from, failureReason);
     }
@@ -84,7 +84,7 @@ public class BatchlogResponseHandler<T> extends AbstractWriteResponseHandler<T>
         return wrapped.totalEndpoints();
     }
 
-    protected boolean waitingFor(InetAddress from)
+    protected boolean waitingFor(InetAddressAndPort from)
     {
         return wrapped.waitingFor(from);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 51219e6..130f3fd 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -46,6 +46,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -231,6 +232,8 @@ public class CassandraDaemon
             }
         });
 
+        SystemKeyspaceMigrator40.migrate();
+
         // Populate token metadata before flushing, for token-aware sstable partitioning (#6696)
         StorageService.instance.populateTokenMetadata();
 
@@ -377,7 +380,7 @@ public class CassandraDaemon
 
         ScheduledExecutors.optionalTasks.schedule(viewRebuild, StorageService.RING_DELAY, TimeUnit.MILLISECONDS);
 
-        if (!FBUtilities.getBroadcastAddress().equals(InetAddress.getLoopbackAddress()))
+        if (!FBUtilities.getBroadcastAddressAndPort().equals(InetAddressAndPort.getLoopbackAddress()))
             Gossiper.waitToSettle();
 
         // re-enable auto-compaction after gossip is settled, so correct disk boundaries are used
@@ -445,7 +448,7 @@ public class CassandraDaemon
     	{
 	        try
 	        {
-	            logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName());
+	            logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName() + ":" + DatabaseDescriptor.getStoragePort() + ":" + DatabaseDescriptor.getSSLStoragePort());
 	        }
 	        catch (UnknownHostException e1)
 	        {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index e41cc4f..97b6172 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -60,7 +60,7 @@ public class ClientState
     {
         // We want these system cfs to be always readable to authenticated users since many tools rely on them
         // (nodetool, cqlsh, bulkloader, etc.)
-        for (String cf : Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.PEERS))
+        for (String cf : Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.LEGACY_PEERS, SystemKeyspace.PEERS_V2))
             READABLE_SYSTEM_RESOURCES.add(DataResource.table(SchemaConstants.SYSTEM_KEYSPACE_NAME, cf));
 
         SchemaKeyspace.ALL.forEach(table -> READABLE_SYSTEM_RESOURCES.add(DataResource.table(SchemaConstants.SCHEMA_KEYSPACE_NAME, table)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/DataResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DataResolver.java b/src/java/org/apache/cassandra/service/DataResolver.java
index 54f4b0c..82db754 100644
--- a/src/java/org/apache/cassandra/service/DataResolver.java
+++ b/src/java/org/apache/cassandra/service/DataResolver.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.TimeoutException;
 
@@ -40,6 +39,7 @@ import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.ExcludingBounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.*;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
@@ -87,7 +87,7 @@ public class DataResolver extends ResponseResolver
         // at the beginning of this method), so grab the response count once and use that through the method.
         int count = responses.size();
         List<UnfilteredPartitionIterator> iters = new ArrayList<>(count);
-        InetAddress[] sources = new InetAddress[count];
+        InetAddressAndPort[] sources = new InetAddressAndPort[count];
         for (int i = 0; i < count; i++)
         {
             MessageIn<ReadResponse> msg = responses.get(i);
@@ -120,7 +120,7 @@ public class DataResolver extends ResponseResolver
     }
 
     private UnfilteredPartitionIterator mergeWithShortReadProtection(List<UnfilteredPartitionIterator> results,
-                                                                     InetAddress[] sources,
+                                                                     InetAddressAndPort[] sources,
                                                                      DataLimits.Counter mergedResultCounter)
     {
         // If we have only one results, there is no read repair to do and we can't get short reads
@@ -140,9 +140,9 @@ public class DataResolver extends ResponseResolver
 
     private class RepairMergeListener implements UnfilteredPartitionIterators.MergeListener
     {
-        private final InetAddress[] sources;
+        private final InetAddressAndPort[] sources;
 
-        private RepairMergeListener(InetAddress[] sources)
+        private RepairMergeListener(InetAddressAndPort[] sources)
         {
             this.sources = sources;
         }
@@ -471,7 +471,7 @@ public class DataResolver extends ResponseResolver
                         sendRepairMutation(repairs[i].build(), sources[i]);
             }
 
-            private void sendRepairMutation(PartitionUpdate partition, InetAddress destination)
+            private void sendRepairMutation(PartitionUpdate partition, InetAddressAndPort destination)
             {
                 Mutation mutation = new Mutation(partition);
                 int messagingVersion = MessagingService.instance().getVersion(destination);
@@ -514,7 +514,7 @@ public class DataResolver extends ResponseResolver
     }
 
     private UnfilteredPartitionIterator extendWithShortReadProtection(UnfilteredPartitionIterator partitions,
-                                                                      InetAddress source,
+                                                                      InetAddressAndPort source,
                                                                       DataLimits.Counter mergedResultCounter)
     {
         DataLimits.Counter singleResultCounter =
@@ -557,7 +557,7 @@ public class DataResolver extends ResponseResolver
      */
     private class ShortReadPartitionsProtection extends Transformation<UnfilteredRowIterator> implements MorePartitions<UnfilteredPartitionIterator>
     {
-        private final InetAddress source;
+        private final InetAddressAndPort source;
 
         private final DataLimits.Counter singleResultCounter; // unmerged per-source counter
         private final DataLimits.Counter mergedResultCounter; // merged end-result counter
@@ -568,7 +568,7 @@ public class DataResolver extends ResponseResolver
 
         private final long queryStartNanoTime;
 
-        private ShortReadPartitionsProtection(InetAddress source,
+        private ShortReadPartitionsProtection(InetAddressAndPort source,
                                               DataLimits.Counter singleResultCounter,
                                               DataLimits.Counter mergedResultCounter,
                                               long queryStartNanoTime)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
index 4137e3a..dbd3667 100644
--- a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
@@ -26,6 +25,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.db.ConsistencyLevel;
@@ -41,8 +41,8 @@ public class DatacenterSyncWriteResponseHandler<T> extends AbstractWriteResponse
     private final Map<String, AtomicInteger> responses = new HashMap<String, AtomicInteger>();
     private final AtomicInteger acks = new AtomicInteger(0);
 
-    public DatacenterSyncWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
-                                              Collection<InetAddress> pendingEndpoints,
+    public DatacenterSyncWriteResponseHandler(Collection<InetAddressAndPort> naturalEndpoints,
+                                              Collection<InetAddressAndPort> pendingEndpoints,
                                               ConsistencyLevel consistencyLevel,
                                               Keyspace keyspace,
                                               Runnable callback,
@@ -63,7 +63,7 @@ public class DatacenterSyncWriteResponseHandler<T> extends AbstractWriteResponse
 
         // During bootstrap, we have to include the pending endpoints or we may fail the consistency level
         // guarantees (see #833)
-        for (InetAddress pending : pendingEndpoints)
+        for (InetAddressAndPort pending : pendingEndpoints)
         {
             responses.get(snitch.getDatacenter(pending)).incrementAndGet();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
index 83dddcf..a8d7b28 100644
--- a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
@@ -17,10 +17,10 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.Collection;
 
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
@@ -30,8 +30,8 @@ import org.apache.cassandra.db.WriteType;
  */
 public class DatacenterWriteResponseHandler<T> extends WriteResponseHandler<T>
 {
-    public DatacenterWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
-                                          Collection<InetAddress> pendingEndpoints,
+    public DatacenterWriteResponseHandler(Collection<InetAddressAndPort> naturalEndpoints,
+                                          Collection<InetAddressAndPort> pendingEndpoints,
                                           ConsistencyLevel consistencyLevel,
                                           Keyspace keyspace,
                                           Runnable callback,
@@ -66,7 +66,7 @@ public class DatacenterWriteResponseHandler<T> extends WriteResponseHandler<T>
     }
 
     @Override
-    protected boolean waitingFor(InetAddress from)
+    protected boolean waitingFor(InetAddressAndPort from)
     {
         return consistencyLevel.isLocal(from);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/IEndpointLifecycleSubscriber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/IEndpointLifecycleSubscriber.java b/src/java/org/apache/cassandra/service/IEndpointLifecycleSubscriber.java
index 24cb3d7..bc49d3b 100644
--- a/src/java/org/apache/cassandra/service/IEndpointLifecycleSubscriber.java
+++ b/src/java/org/apache/cassandra/service/IEndpointLifecycleSubscriber.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Interface on which interested parties can be notified of high level endpoint
@@ -35,33 +35,33 @@ public interface IEndpointLifecycleSubscriber
      *
      * @param endpoint the newly added endpoint.
      */
-    public void onJoinCluster(InetAddress endpoint);
+    public void onJoinCluster(InetAddressAndPort endpoint);
 
     /**
      * Called when a new node leave the cluster (decommission or removeToken).
      *
      * @param endpoint the endpoint that is leaving.
      */
-    public void onLeaveCluster(InetAddress endpoint);
+    public void onLeaveCluster(InetAddressAndPort endpoint);
 
     /**
      * Called when a node is marked UP.
      *
      * @param endpoint the endpoint marked UP.
      */
-    public void onUp(InetAddress endpoint);
+    public void onUp(InetAddressAndPort endpoint);
 
     /**
      * Called when a node is marked DOWN.
      *
      * @param endpoint the endpoint marked DOWN.
      */
-    public void onDown(InetAddress endpoint);
+    public void onDown(InetAddressAndPort endpoint);
 
     /**
      * Called when a node has moved (to a new token).
      *
      * @param endpoint the endpoint that has moved.
      */
-    public void onMove(InetAddress endpoint);
+    public void onMove(InetAddressAndPort endpoint);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/LoadBroadcaster.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/LoadBroadcaster.java b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
index 945dd2f..35c0b62 100644
--- a/src/java/org/apache/cassandra/service/LoadBroadcaster.java
+++ b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
@@ -17,12 +17,12 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,21 +38,21 @@ public class LoadBroadcaster implements IEndpointStateChangeSubscriber
 
     private static final Logger logger = LoggerFactory.getLogger(LoadBroadcaster.class);
 
-    private ConcurrentMap<InetAddress, Double> loadInfo = new ConcurrentHashMap<InetAddress, java.lang.Double>();
+    private ConcurrentMap<InetAddressAndPort, Double> loadInfo = new ConcurrentHashMap<>();
 
     private LoadBroadcaster()
     {
         Gossiper.instance.register(this);
     }
 
-    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value)
+    public void onChange(InetAddressAndPort endpoint, ApplicationState state, VersionedValue value)
     {
         if (state != ApplicationState.LOAD)
             return;
         loadInfo.put(endpoint, Double.valueOf(value.value));
     }
 
-    public void onJoin(InetAddress endpoint, EndpointState epState)
+    public void onJoin(InetAddressAndPort endpoint, EndpointState epState)
     {
         VersionedValue localValue = epState.getApplicationState(ApplicationState.LOAD);
         if (localValue != null)
@@ -61,20 +61,20 @@ public class LoadBroadcaster implements IEndpointStateChangeSubscriber
         }
     }
     
-    public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
+    public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
 
-    public void onAlive(InetAddress endpoint, EndpointState state) {}
+    public void onAlive(InetAddressAndPort endpoint, EndpointState state) {}
 
-    public void onDead(InetAddress endpoint, EndpointState state) {}
+    public void onDead(InetAddressAndPort endpoint, EndpointState state) {}
 
-    public void onRestart(InetAddress endpoint, EndpointState state) {}
+    public void onRestart(InetAddressAndPort endpoint, EndpointState state) {}
 
-    public void onRemove(InetAddress endpoint)
+    public void onRemove(InetAddressAndPort endpoint)
     {
         loadInfo.remove(endpoint);
     }
 
-    public Map<InetAddress, Double> getLoadInfo()
+    public Map<InetAddressAndPort, Double> getLoadInfo()
     {
         return Collections.unmodifiableMap(loadInfo);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/ReadCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ReadCallback.java b/src/java/org/apache/cassandra/service/ReadCallback.java
index 7ee6386..e7f30b4 100644
--- a/src/java/org/apache/cassandra/service/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/ReadCallback.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -38,6 +37,7 @@ import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
@@ -56,7 +56,7 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
     final SimpleCondition condition = new SimpleCondition();
     private final long queryStartNanoTime;
     final int blockfor;
-    final List<InetAddress> endpoints;
+    final List<InetAddressAndPort> endpoints;
     private final ReadCommand command;
     private final ConsistencyLevel consistencyLevel;
     private static final AtomicIntegerFieldUpdater<ReadCallback> recievedUpdater
@@ -65,14 +65,14 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
     private static final AtomicIntegerFieldUpdater<ReadCallback> failuresUpdater
             = AtomicIntegerFieldUpdater.newUpdater(ReadCallback.class, "failures");
     private volatile int failures = 0;
-    private final Map<InetAddress, RequestFailureReason> failureReasonByEndpoint;
+    private final Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint;
 
     private final Keyspace keyspace; // TODO push this into ConsistencyLevel?
 
     /**
      * Constructor when response count has to be calculated and blocked for.
      */
-    public ReadCallback(ResponseResolver resolver, ConsistencyLevel consistencyLevel, ReadCommand command, List<InetAddress> filteredEndpoints, long queryStartNanoTime)
+    public ReadCallback(ResponseResolver resolver, ConsistencyLevel consistencyLevel, ReadCommand command, List<InetAddressAndPort> filteredEndpoints, long queryStartNanoTime)
     {
         this(resolver,
              consistencyLevel,
@@ -83,7 +83,7 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
              queryStartNanoTime);
     }
 
-    public ReadCallback(ResponseResolver resolver, ConsistencyLevel consistencyLevel, int blockfor, ReadCommand command, Keyspace keyspace, List<InetAddress> endpoints, long queryStartNanoTime)
+    public ReadCallback(ResponseResolver resolver, ConsistencyLevel consistencyLevel, int blockfor, ReadCommand command, Keyspace keyspace, List<InetAddressAndPort> endpoints, long queryStartNanoTime)
     {
         this.command = command;
         this.keyspace = keyspace;
@@ -176,7 +176,7 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
     /**
      * @return true if the message counts towards the blockfor threshold
      */
-    private boolean waitingFor(InetAddress from)
+    private boolean waitingFor(InetAddressAndPort from)
     {
         return consistencyLevel.isDatacenterLocal()
              ? DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(from))
@@ -193,9 +193,9 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
 
     public void response(ReadResponse result)
     {
-        MessageIn<ReadResponse> message = MessageIn.create(FBUtilities.getBroadcastAddress(),
+        MessageIn<ReadResponse> message = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(),
                                                            result,
-                                                           Collections.<String, byte[]>emptyMap(),
+                                                           Collections.emptyMap(),
                                                            MessagingService.Verb.INTERNAL_RESPONSE,
                                                            MessagingService.current_version);
         response(message);
@@ -245,14 +245,14 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
                 final DataResolver repairResolver = new DataResolver(keyspace, command, consistencyLevel, endpoints.size(), queryStartNanoTime);
                 AsyncRepairCallback repairHandler = new AsyncRepairCallback(repairResolver, endpoints.size());
 
-                for (InetAddress endpoint : endpoints)
+                for (InetAddressAndPort endpoint : endpoints)
                     MessagingService.instance().sendRR(command.createMessage(), endpoint, repairHandler);
             }
         }
     }
 
     @Override
-    public void onFailure(InetAddress from, RequestFailureReason failureReason)
+    public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
     {
         int n = waitingFor(from)
               ? failuresUpdater.incrementAndGet(this)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index 55099fc..224fd5e 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -437,7 +437,7 @@ public class StartupChecks
                 String storedDc = SystemKeyspace.getDatacenter();
                 if (storedDc != null)
                 {
-                    String currentDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+                    String currentDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
                     if (!storedDc.equals(currentDc))
                     {
                         String formatMessage = "Cannot start node if snitch's data center (%s) differs from previous data center (%s). " +
@@ -459,7 +459,7 @@ public class StartupChecks
                 String storedRack = SystemKeyspace.getRack();
                 if (storedRack != null)
                 {
-                    String currentRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress());
+                    String currentRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddressAndPort());
                     if (!storedRack.equals(currentRack))
                     {
                         String formatMessage = "Cannot start node if snitch's rack (%s) differs from previous rack (%s). " +


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


[12/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/asymmetric/ReduceHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/ReduceHelper.java b/src/java/org/apache/cassandra/repair/asymmetric/ReduceHelper.java
index ce05e93..c7d45bf 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/ReduceHelper.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/ReduceHelper.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.asymmetric;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -31,6 +30,7 @@ import com.google.common.collect.ImmutableMap;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Basic idea is that we track incoming ranges instead of blindly just exchanging the ranges that mismatch between two nodes
@@ -47,19 +47,19 @@ public class ReduceHelper
     /**
      * Reduces the differences provided by the merkle trees to a minimum set of differences
      */
-    public static ImmutableMap<InetAddress, HostDifferences> reduce(DifferenceHolder differences, PreferedNodeFilter filter)
+    public static ImmutableMap<InetAddressAndPort, HostDifferences> reduce(DifferenceHolder differences, PreferedNodeFilter filter)
     {
-        Map<InetAddress, IncomingRepairStreamTracker> trackers = createIncomingRepairStreamTrackers(differences);
-        Map<InetAddress, Integer> outgoingStreamCounts = new HashMap<>();
-        ImmutableMap.Builder<InetAddress, HostDifferences> mapBuilder = ImmutableMap.builder();
-        for (Map.Entry<InetAddress, IncomingRepairStreamTracker> trackerEntry : trackers.entrySet())
+        Map<InetAddressAndPort, IncomingRepairStreamTracker> trackers = createIncomingRepairStreamTrackers(differences);
+        Map<InetAddressAndPort, Integer> outgoingStreamCounts = new HashMap<>();
+        ImmutableMap.Builder<InetAddressAndPort, HostDifferences> mapBuilder = ImmutableMap.builder();
+        for (Map.Entry<InetAddressAndPort, IncomingRepairStreamTracker> trackerEntry : trackers.entrySet())
         {
             IncomingRepairStreamTracker tracker = trackerEntry.getValue();
             HostDifferences rangesToFetch = new HostDifferences();
             for (Map.Entry<Range<Token>, StreamFromOptions> entry : tracker.getIncoming().entrySet())
             {
                 Range<Token> rangeToFetch = entry.getKey();
-                for (InetAddress remoteNode : pickLeastStreaming(trackerEntry.getKey(), entry.getValue(), outgoingStreamCounts, filter))
+                for (InetAddressAndPort remoteNode : pickLeastStreaming(trackerEntry.getKey(), entry.getValue(), outgoingStreamCounts, filter))
                     rangesToFetch.addSingleRange(remoteNode, rangeToFetch);
             }
             mapBuilder.put(trackerEntry.getKey(), rangesToFetch);
@@ -69,14 +69,14 @@ public class ReduceHelper
     }
 
     @VisibleForTesting
-    static Map<InetAddress, IncomingRepairStreamTracker> createIncomingRepairStreamTrackers(DifferenceHolder differences)
+    static Map<InetAddressAndPort, IncomingRepairStreamTracker> createIncomingRepairStreamTrackers(DifferenceHolder differences)
     {
-        Map<InetAddress, IncomingRepairStreamTracker> trackers = new HashMap<>();
+        Map<InetAddressAndPort, IncomingRepairStreamTracker> trackers = new HashMap<>();
 
-        for (InetAddress hostWithDifference : differences.keyHosts())
+        for (InetAddressAndPort hostWithDifference : differences.keyHosts())
         {
             HostDifferences hostDifferences = differences.get(hostWithDifference);
-            for (InetAddress differingHost : hostDifferences.hosts())
+            for (InetAddressAndPort differingHost : hostDifferences.hosts())
             {
                 List<Range<Token>> differingRanges = hostDifferences.get(differingHost);
                 // hostWithDifference has mismatching ranges differingRanges with differingHost:
@@ -93,24 +93,24 @@ public class ReduceHelper
     }
 
     private static IncomingRepairStreamTracker getTracker(DifferenceHolder differences,
-                                                          Map<InetAddress, IncomingRepairStreamTracker> trackers,
-                                                          InetAddress host)
+                                                          Map<InetAddressAndPort, IncomingRepairStreamTracker> trackers,
+                                                          InetAddressAndPort host)
     {
         return trackers.computeIfAbsent(host, (h) -> new IncomingRepairStreamTracker(differences));
     }
 
     // greedily pick the nodes doing the least amount of streaming
-    private static Collection<InetAddress> pickLeastStreaming(InetAddress streamingNode,
+    private static Collection<InetAddressAndPort> pickLeastStreaming(InetAddressAndPort streamingNode,
                                                               StreamFromOptions toStreamFrom,
-                                                              Map<InetAddress, Integer> outgoingStreamCounts,
+                                                              Map<InetAddressAndPort, Integer> outgoingStreamCounts,
                                                               PreferedNodeFilter filter)
     {
-        Set<InetAddress> retSet = new HashSet<>();
-        for (Set<InetAddress> toStream : toStreamFrom.allStreams())
+        Set<InetAddressAndPort> retSet = new HashSet<>();
+        for (Set<InetAddressAndPort> toStream : toStreamFrom.allStreams())
         {
-            InetAddress candidate = null;
-            Set<InetAddress> prefered = filter.apply(streamingNode, toStream);
-            for (InetAddress node : prefered)
+            InetAddressAndPort candidate = null;
+            Set<InetAddressAndPort> prefered = filter.apply(streamingNode, toStream);
+            for (InetAddressAndPort node : prefered)
             {
                 if (candidate == null || outgoingStreamCounts.getOrDefault(candidate, 0) > outgoingStreamCounts.getOrDefault(node, 0))
                 {
@@ -120,7 +120,7 @@ public class ReduceHelper
             // ok, found no prefered hosts, try all of them
             if (candidate == null)
             {
-                for (InetAddress node : toStream)
+                for (InetAddressAndPort node : toStream)
                 {
                     if (candidate == null || outgoingStreamCounts.getOrDefault(candidate, 0) > outgoingStreamCounts.getOrDefault(node, 0))
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/asymmetric/StreamFromOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/StreamFromOptions.java b/src/java/org/apache/cassandra/repair/asymmetric/StreamFromOptions.java
index 4516f23..6070983 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/StreamFromOptions.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/StreamFromOptions.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.asymmetric;
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
@@ -28,6 +27,7 @@ import com.google.common.collect.Sets;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Keeps track of where a node needs to stream a given range from.
@@ -53,18 +53,18 @@ public class StreamFromOptions
     /**
      * Contains the hosts to stream from - if two nodes are in the same inner set, they are identical for the range we are handling
      */
-    private final Set<Set<InetAddress>> streamOptions = new HashSet<>();
+    private final Set<Set<InetAddressAndPort>> streamOptions = new HashSet<>();
 
     public StreamFromOptions(DifferenceHolder differences, Range<Token> range)
     {
         this(differences, range, Collections.emptySet());
     }
 
-    private StreamFromOptions(DifferenceHolder differences, Range<Token> range, Set<Set<InetAddress>> existing)
+    private StreamFromOptions(DifferenceHolder differences, Range<Token> range, Set<Set<InetAddressAndPort>> existing)
     {
         this.differences = differences;
         this.range = range;
-        for (Set<InetAddress> addresses : existing)
+        for (Set<InetAddressAndPort> addresses : existing)
             this.streamOptions.add(Sets.newHashSet(addresses));
     }
 
@@ -75,11 +75,11 @@ public class StreamFromOptions
      * range we are tracking, then just add it to the set with the identical remote nodes. Otherwise create a new group
      * of nodes containing this new node.
      */
-    public void add(InetAddress streamFromNode)
+    public void add(InetAddressAndPort streamFromNode)
     {
-        for (Set<InetAddress> options : streamOptions)
+        for (Set<InetAddressAndPort> options : streamOptions)
         {
-            InetAddress first = options.iterator().next();
+            InetAddressAndPort first = options.iterator().next();
             if (!differences.hasDifferenceBetween(first, streamFromNode, range))
             {
                 options.add(streamFromNode);
@@ -94,7 +94,7 @@ public class StreamFromOptions
         return new StreamFromOptions(differences, withRange, streamOptions);
     }
 
-    public Iterable<Set<InetAddress>> allStreams()
+    public Iterable<Set<InetAddressAndPort>> allStreams()
     {
         return streamOptions;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java b/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java
index c137346..78057e2 100644
--- a/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java
+++ b/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java
@@ -33,6 +33,7 @@ import com.google.common.collect.Iterables;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.messages.FailSession;
 import org.apache.cassandra.repair.messages.FinalizeCommit;
 import org.apache.cassandra.repair.messages.FinalizePromise;
@@ -55,7 +56,7 @@ import org.apache.cassandra.tools.nodetool.RepairAdmin;
  * There are 4 stages to a consistent incremental repair.
  *
  * <h1>Repair prepare</h1>
- *  First, the normal {@link ActiveRepairService#prepareForRepair(UUID, InetAddress, Set, RepairOption, List)} stuff
+ *  First, the normal {@link ActiveRepairService#prepareForRepair(UUID, InetAddressAndPort, Set, RepairOption, List)} stuff
  *  happens, which sends out {@link PrepareMessage} and creates a {@link ActiveRepairService.ParentRepairSession}
  *  on the coordinator and each of the neighbors.
  *
@@ -68,7 +69,7 @@ import org.apache.cassandra.tools.nodetool.RepairAdmin;
  *  coordinator indicating success or failure. If the pending anti-compaction fails, the local session state is set
  *  to {@code FAILED}.
  *  <p/>
- *  (see {@link LocalSessions#handlePrepareMessage(InetAddress, PrepareConsistentRequest)}
+ *  (see {@link LocalSessions#handlePrepareMessage(InetAddressAndPort, PrepareConsistentRequest)}
  *  <p/>
  *  Once the coordinator recieves positive {@code PrepareConsistentResponse} messages from all the participants, the
  *  coordinator begins the normal repair process.
@@ -99,8 +100,8 @@ import org.apache.cassandra.tools.nodetool.RepairAdmin;
  *  & {@link CoordinatorSession#finalizeCommit()}
  *  <p/>
  *
- *  On the local session side, see {@link LocalSessions#handleFinalizeProposeMessage(InetAddress, FinalizePropose)}
- *  & {@link LocalSessions#handleFinalizeCommitMessage(InetAddress, FinalizeCommit)}
+ *  On the local session side, see {@link LocalSessions#handleFinalizeProposeMessage(InetAddressAndPort, FinalizePropose)}
+ *  & {@link LocalSessions#handleFinalizeCommitMessage(InetAddressAndPort, FinalizeCommit)}
  *
  * <h1>Failure</h1>
  *  If there are any failures or problems during the process above, the session will be failed. When a session is failed,
@@ -187,11 +188,11 @@ public abstract class ConsistentSession
 
     private volatile State state;
     public final UUID sessionID;
-    public final InetAddress coordinator;
+    public final InetAddressAndPort coordinator;
     public final ImmutableSet<TableId> tableIds;
     public final long repairedAt;
     public final ImmutableSet<Range<Token>> ranges;
-    public final ImmutableSet<InetAddress> participants;
+    public final ImmutableSet<InetAddressAndPort> participants;
 
     ConsistentSession(AbstractBuilder builder)
     {
@@ -260,11 +261,11 @@ public abstract class ConsistentSession
     {
         private State state;
         private UUID sessionID;
-        private InetAddress coordinator;
+        private InetAddressAndPort coordinator;
         private Set<TableId> ids;
         private long repairedAt;
         private Collection<Range<Token>> ranges;
-        private Set<InetAddress> participants;
+        private Set<InetAddressAndPort> participants;
 
         void withState(State state)
         {
@@ -276,7 +277,7 @@ public abstract class ConsistentSession
             this.sessionID = sessionID;
         }
 
-        void withCoordinator(InetAddress coordinator)
+        void withCoordinator(InetAddressAndPort coordinator)
         {
             this.coordinator = coordinator;
         }
@@ -301,7 +302,7 @@ public abstract class ConsistentSession
             this.ranges = ranges;
         }
 
-        void withParticipants(Set<InetAddress> peers)
+        void withParticipants(Set<InetAddressAndPort> peers)
         {
             this.participants = peers;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java
index ba0025f..f52a28d 100644
--- a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java
+++ b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -39,6 +38,7 @@ import org.apache.commons.lang3.time.DurationFormatUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.RepairSessionResult;
@@ -58,7 +58,7 @@ public class CoordinatorSession extends ConsistentSession
 {
     private static final Logger logger = LoggerFactory.getLogger(CoordinatorSession.class);
 
-    private final Map<InetAddress, State> participantStates = new HashMap<>();
+    private final Map<InetAddressAndPort, State> participantStates = new HashMap<>();
     private final SettableFuture<Boolean> prepareFuture = SettableFuture.create();
     private final SettableFuture<Boolean> finalizeProposeFuture = SettableFuture.create();
 
@@ -69,7 +69,7 @@ public class CoordinatorSession extends ConsistentSession
     public CoordinatorSession(Builder builder)
     {
         super(builder);
-        for (InetAddress participant : participants)
+        for (InetAddressAndPort participant : participants)
         {
             participantStates.put(participant, State.PREPARING);
         }
@@ -95,7 +95,7 @@ public class CoordinatorSession extends ConsistentSession
         super.setState(state);
     }
 
-    public synchronized void setParticipantState(InetAddress participant, State state)
+    public synchronized void setParticipantState(InetAddressAndPort participant, State state)
     {
         logger.trace("Setting participant {} to state {} for repair {}", participant, state, sessionID);
         Preconditions.checkArgument(participantStates.containsKey(participant),
@@ -115,7 +115,7 @@ public class CoordinatorSession extends ConsistentSession
 
     synchronized void setAll(State state)
     {
-        for (InetAddress participant : participants)
+        for (InetAddressAndPort participant : participants)
         {
             setParticipantState(participant, state);
         }
@@ -131,7 +131,7 @@ public class CoordinatorSession extends ConsistentSession
         return getState() == State.FAILED || Iterables.any(participantStates.values(), v -> v == State.FAILED);
     }
 
-    protected void sendMessage(InetAddress destination, RepairMessage message)
+    protected void sendMessage(InetAddressAndPort destination, RepairMessage message)
     {
         logger.trace("Sending {} to {}", message, destination);
         MessageOut<RepairMessage> messageOut = new MessageOut<RepairMessage>(MessagingService.Verb.REPAIR_MESSAGE, message, RepairMessage.serializer);
@@ -144,14 +144,14 @@ public class CoordinatorSession extends ConsistentSession
 
         logger.debug("Beginning prepare phase of incremental repair session {}", sessionID);
         PrepareConsistentRequest message = new PrepareConsistentRequest(sessionID, coordinator, participants);
-        for (final InetAddress participant : participants)
+        for (final InetAddressAndPort participant : participants)
         {
             sendMessage(participant, message);
         }
         return prepareFuture;
     }
 
-    public synchronized void handlePrepareResponse(InetAddress participant, boolean success)
+    public synchronized void handlePrepareResponse(InetAddressAndPort participant, boolean success)
     {
         if (getState() == State.FAILED)
         {
@@ -185,14 +185,14 @@ public class CoordinatorSession extends ConsistentSession
         Preconditions.checkArgument(allStates(State.REPAIRING));
         logger.debug("Proposing finalization of repair session {}", sessionID);
         FinalizePropose message = new FinalizePropose(sessionID);
-        for (final InetAddress participant : participants)
+        for (final InetAddressAndPort participant : participants)
         {
             sendMessage(participant, message);
         }
         return finalizeProposeFuture;
     }
 
-    public synchronized void handleFinalizePromise(InetAddress participant, boolean success)
+    public synchronized void handleFinalizePromise(InetAddressAndPort participant, boolean success)
     {
         if (getState() == State.FAILED)
         {
@@ -221,7 +221,7 @@ public class CoordinatorSession extends ConsistentSession
         Preconditions.checkArgument(allStates(State.FINALIZE_PROMISED));
         logger.debug("Committing finalization of repair session {}", sessionID);
         FinalizeCommit message = new FinalizeCommit(sessionID);
-        for (final InetAddress participant : participants)
+        for (final InetAddressAndPort participant : participants)
         {
             sendMessage(participant, message);
         }
@@ -233,7 +233,7 @@ public class CoordinatorSession extends ConsistentSession
     {
         logger.info("Incremental repair session {} failed", sessionID);
         FailSession message = new FailSession(sessionID);
-        for (final InetAddress participant : participants)
+        for (final InetAddressAndPort participant : participants)
         {
             if (participantStates.get(participant) != State.FAILED)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/consistent/CoordinatorSessions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSessions.java b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSessions.java
index 211e0c1..bb84d0a 100644
--- a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSessions.java
+++ b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSessions.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
@@ -26,6 +25,7 @@ import java.util.UUID;
 
 import com.google.common.base.Preconditions;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.messages.FailSession;
 import org.apache.cassandra.repair.messages.FinalizePromise;
 import org.apache.cassandra.repair.messages.PrepareConsistentResponse;
@@ -43,7 +43,7 @@ public class CoordinatorSessions
         return new CoordinatorSession(builder);
     }
 
-    public synchronized CoordinatorSession registerSession(UUID sessionId, Set<InetAddress> participants)
+    public synchronized CoordinatorSession registerSession(UUID sessionId, Set<InetAddressAndPort> participants)
     {
         Preconditions.checkArgument(!sessions.containsKey(sessionId), "A coordinator already exists for session %s", sessionId);
         ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(sessionId);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java
index 903aeb5..98b883a 100644
--- a/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java
+++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java
@@ -21,10 +21,12 @@ package org.apache.cassandra.repair.consistent;
 import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Iterables;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
@@ -40,6 +42,7 @@ public class LocalSessionInfo
     public static final String LAST_UPDATE = "LAST_UPDATE";
     public static final String COORDINATOR = "COORDINATOR";
     public static final String PARTICIPANTS = "PARTICIPANTS";
+    public static final String PARTICIPANTS_WP = "PARTICIPANTS_WP";
     public static final String TABLES = "TABLES";
 
 
@@ -59,7 +62,8 @@ public class LocalSessionInfo
         m.put(STARTED, Integer.toString(session.getStartedAt()));
         m.put(LAST_UPDATE, Integer.toString(session.getLastUpdate()));
         m.put(COORDINATOR, session.coordinator.toString());
-        m.put(PARTICIPANTS, Joiner.on(',').join(Iterables.transform(session.participants, InetAddress::toString)));
+        m.put(PARTICIPANTS, Joiner.on(',').join(Iterables.transform(session.participants.stream().map(peer -> peer.address).collect(Collectors.toList()), InetAddress::getHostAddress)));
+        m.put(PARTICIPANTS_WP, Joiner.on(',').join(Iterables.transform(session.participants, InetAddressAndPort::toString)));
         m.put(TABLES, Joiner.on(',').join(Iterables.transform(session.tableIds, LocalSessionInfo::tableString)));
 
         return m;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
index 4ef2c2c..e62f6fd 100644
--- a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
+++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.repair.consistent;
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.time.Instant;
 import java.util.Date;
@@ -33,6 +34,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
+import java.util.stream.Collectors;
 import javax.annotation.Nullable;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -49,13 +51,18 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.InetAddressType;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
@@ -141,13 +148,13 @@ public class LocalSessions
     }
 
     @VisibleForTesting
-    protected InetAddress getBroadcastAddress()
+    protected InetAddressAndPort getBroadcastAddressAndPort()
     {
-        return FBUtilities.getBroadcastAddress();
+        return FBUtilities.getBroadcastAddressAndPort();
     }
 
     @VisibleForTesting
-    protected boolean isAlive(InetAddress address)
+    protected boolean isAlive(InetAddressAndPort address)
     {
         return FailureDetector.instance.isAlive(address);
     }
@@ -177,14 +184,14 @@ public class LocalSessions
         logger.info("Cancelling local repair session {}", sessionID);
         LocalSession session = getSession(sessionID);
         Preconditions.checkArgument(session != null, "Session {} does not exist", sessionID);
-        Preconditions.checkArgument(force || session.coordinator.equals(getBroadcastAddress()),
+        Preconditions.checkArgument(force || session.coordinator.equals(getBroadcastAddressAndPort()),
                                     "Cancel session %s from it's coordinator (%s) or use --force",
                                     sessionID, session.coordinator);
 
         setStateAndSave(session, FAILED);
-        for (InetAddress participant : session.participants)
+        for (InetAddressAndPort participant : session.participants)
         {
-            if (!participant.equals(getBroadcastAddress()))
+            if (!participant.equals(getBroadcastAddressAndPort()))
                 sendMessage(participant, new FailSession(sessionID));
         }
     }
@@ -335,10 +342,12 @@ public class LocalSessions
                        "repaired_at, " +
                        "state, " +
                        "coordinator, " +
+                       "coordinator_port, " +
                        "participants, " +
+                       "participants_wp," +
                        "ranges, " +
                        "cfids) " +
-                       "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                       "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
         QueryProcessor.executeInternal(String.format(query, keyspace, table),
                                        session.sessionID,
@@ -346,8 +355,10 @@ public class LocalSessions
                                        Date.from(Instant.ofEpochSecond(session.getLastUpdate())),
                                        Date.from(Instant.ofEpochMilli(session.repairedAt)),
                                        session.getState().ordinal(),
-                                       session.coordinator,
-                                       session.participants,
+                                       session.coordinator.address,
+                                       session.coordinator.port,
+                                       session.participants.stream().map(participant -> participant.address).collect(Collectors.toSet()),
+                                       session.participants.stream().map(participant -> participant.toString()).collect(Collectors.toSet()),
                                        serializeRanges(session.ranges),
                                        tableIdToUuid(session.tableIds));
     }
@@ -362,12 +373,27 @@ public class LocalSessions
         LocalSession.Builder builder = LocalSession.builder();
         builder.withState(ConsistentSession.State.valueOf(row.getInt("state")));
         builder.withSessionID(row.getUUID("parent_id"));
-        builder.withCoordinator(row.getInetAddress("coordinator"));
+        InetAddressAndPort coordinator = InetAddressAndPort.getByAddressOverrideDefaults(
+            row.getInetAddress("coordinator"),
+            row.getInt("coordinator_port"));
+        builder.withCoordinator(coordinator);
         builder.withTableIds(uuidToTableId(row.getSet("cfids", UUIDType.instance)));
         builder.withRepairedAt(row.getTimestamp("repaired_at").getTime());
         builder.withRanges(deserializeRanges(row.getSet("ranges", BytesType.instance)));
-        builder.withParticipants(row.getSet("participants", InetAddressType.instance));
-
+        //There is no cross version streaming and thus no cross version repair so assume that
+        //any valid repair sessions has the participants_wp column and any that doesn't is malformed
+        Set<String> participants = row.getSet("participants_wp", UTF8Type.instance);
+        builder.withParticipants(participants.stream().map(participant ->
+                                                             {
+                                                                 try
+                                                                 {
+                                                                     return InetAddressAndPort.getByName(participant);
+                                                                 }
+                                                                 catch (UnknownHostException e)
+                                                                 {
+                                                                     throw new RuntimeException(e);
+                                                                 }
+                                                             }).collect(Collectors.toSet()));
         builder.withStartedAt(dateToSeconds(row.getTimestamp("started_at")));
         builder.withLastUpdate(dateToSeconds(row.getTimestamp("last_update")));
 
@@ -440,7 +466,7 @@ public class LocalSessions
     }
 
     @VisibleForTesting
-    LocalSession createSessionUnsafe(UUID sessionId, ActiveRepairService.ParentRepairSession prs, Set<InetAddress> peers)
+    LocalSession createSessionUnsafe(UUID sessionId, ActiveRepairService.ParentRepairSession prs, Set<InetAddressAndPort> peers)
     {
         LocalSession.Builder builder = LocalSession.builder();
         builder.withState(ConsistentSession.State.PREPARING);
@@ -464,7 +490,7 @@ public class LocalSessions
         return ActiveRepairService.instance.getParentRepairSession(sessionID);
     }
 
-    protected void sendMessage(InetAddress destination, RepairMessage message)
+    protected void sendMessage(InetAddressAndPort destination, RepairMessage message)
     {
         logger.trace("sending {} to {}", message, destination);
         MessageOut<RepairMessage> messageOut = new MessageOut<RepairMessage>(MessagingService.Verb.REPAIR_MESSAGE, message, RepairMessage.serializer);
@@ -536,12 +562,12 @@ public class LocalSessions
      * successfully. If the pending anti compaction fails, a failure message is sent to the coordinator,
      * cancelling the session.
      */
-    public void handlePrepareMessage(InetAddress from, PrepareConsistentRequest request)
+    public void handlePrepareMessage(InetAddressAndPort from, PrepareConsistentRequest request)
     {
         logger.trace("received {} from {}", request, from);
         UUID sessionID = request.parentSession;
-        InetAddress coordinator = request.coordinator;
-        Set<InetAddress> peers = request.participants;
+        InetAddressAndPort coordinator = request.coordinator;
+        Set<InetAddressAndPort> peers = request.participants;
 
         ActiveRepairService.ParentRepairSession parentSession;
         try
@@ -568,7 +594,7 @@ public class LocalSessions
             {
                 logger.debug("Prepare phase for incremental repair session {} completed", sessionID);
                 setStateAndSave(session, PREPARED);
-                sendMessage(coordinator, new PrepareConsistentResponse(sessionID, getBroadcastAddress(), true));
+                sendMessage(coordinator, new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), true));
                 executor.shutdown();
             }
 
@@ -587,7 +613,7 @@ public class LocalSessions
                 {
                     logger.error("Prepare phase for incremental repair session {} failed", sessionID, t);
                 }
-                sendMessage(coordinator, new PrepareConsistentResponse(sessionID, getBroadcastAddress(), false));
+                sendMessage(coordinator, new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), false));
                 failSession(sessionID, false);
                 executor.shutdown();
             }
@@ -604,7 +630,7 @@ public class LocalSessions
         }
     }
 
-    public void handleFinalizeProposeMessage(InetAddress from, FinalizePropose propose)
+    public void handleFinalizeProposeMessage(InetAddressAndPort from, FinalizePropose propose)
     {
         logger.trace("received {} from {}", propose, from);
         UUID sessionID = propose.sessionID;
@@ -629,7 +655,7 @@ public class LocalSessions
              */
             syncTable();
 
-            sendMessage(from, new FinalizePromise(sessionID, getBroadcastAddress(), true));
+            sendMessage(from, new FinalizePromise(sessionID, getBroadcastAddressAndPort(), true));
             logger.debug("Received FinalizePropose message for incremental repair session {}, responded with FinalizePromise", sessionID);
         }
         catch (IllegalArgumentException e)
@@ -659,7 +685,7 @@ public class LocalSessions
      * as part of the compaction process, and avoids having to worry about in progress compactions interfering with the
      * promotion.
      */
-    public void handleFinalizeCommitMessage(InetAddress from, FinalizeCommit commit)
+    public void handleFinalizeCommitMessage(InetAddressAndPort from, FinalizeCommit commit)
     {
         logger.trace("received {} from {}", commit, from);
         UUID sessionID = commit.sessionID;
@@ -674,7 +700,7 @@ public class LocalSessions
         logger.info("Finalized local repair session {}", sessionID);
     }
 
-    public void handleFailSessionMessage(InetAddress from, FailSession msg)
+    public void handleFailSessionMessage(InetAddressAndPort from, FailSession msg)
     {
         logger.trace("received {} from {}", msg, from);
         failSession(msg.sessionID, false);
@@ -684,16 +710,16 @@ public class LocalSessions
     {
         logger.debug("Attempting to learn the outcome of unfinished local incremental repair session {}", session.sessionID);
         StatusRequest request = new StatusRequest(session.sessionID);
-        for (InetAddress participant : session.participants)
+        for (InetAddressAndPort participant : session.participants)
         {
-            if (!getBroadcastAddress().equals(participant) && isAlive(participant))
+            if (!getBroadcastAddressAndPort().equals(participant) && isAlive(participant))
             {
                 sendMessage(participant, request);
             }
         }
     }
 
-    public void handleStatusRequest(InetAddress from, StatusRequest request)
+    public void handleStatusRequest(InetAddressAndPort from, StatusRequest request)
     {
         logger.trace("received {} from {}", request, from);
         UUID sessionID = request.sessionID;
@@ -710,7 +736,7 @@ public class LocalSessions
        }
     }
 
-    public void handleStatusResponse(InetAddress from, StatusResponse response)
+    public void handleStatusResponse(InetAddressAndPort from, StatusResponse response)
     {
         logger.trace("received {} from {}", response, from);
         UUID sessionID = response.sessionID;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
index 015b558..156fde7 100644
--- a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
+++ b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
@@ -27,6 +26,7 @@ import java.util.Objects;
 
 import com.google.common.collect.Lists;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.RepairResult;
 import org.apache.cassandra.repair.RepairSessionResult;
 import org.apache.cassandra.repair.SyncStat;
@@ -42,14 +42,14 @@ public class SyncStatSummary
 
     private static class Session
     {
-        final InetAddress src;
-        final InetAddress dst;
+        final InetAddressAndPort src;
+        final InetAddressAndPort dst;
 
         int files = 0;
         long bytes = 0;
         long ranges = 0;
 
-        Session(InetAddress src, InetAddress dst)
+        Session(InetAddressAndPort src, InetAddressAndPort dst)
         {
             this.src = src;
             this.dst = dst;
@@ -84,7 +84,7 @@ public class SyncStatSummary
         int ranges = -1;
         boolean totalsCalculated = false;
 
-        final Map<Pair<InetAddress, InetAddress>, Session> sessions = new HashMap<>();
+        final Map<Pair<InetAddressAndPort, InetAddressAndPort>, Session> sessions = new HashMap<>();
 
         Table(String keyspace, String table)
         {
@@ -92,9 +92,9 @@ public class SyncStatSummary
             this.table = table;
         }
 
-        Session getOrCreate(InetAddress from, InetAddress to)
+        Session getOrCreate(InetAddressAndPort from, InetAddressAndPort to)
         {
-            Pair<InetAddress, InetAddress> k = Pair.create(from, to);
+            Pair<InetAddressAndPort, InetAddressAndPort> k = Pair.create(from, to);
             if (!sessions.containsKey(k))
             {
                 sessions.put(k, new Session(from, to));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java b/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java
index b75ad7f..6d76269 100644
--- a/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java
+++ b/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java
@@ -30,6 +30,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.RepairJobDesc;
@@ -39,13 +40,13 @@ public class AsymmetricSyncRequest extends RepairMessage
 {
     public static MessageSerializer serializer = new SyncRequestSerializer();
 
-    public final InetAddress initiator;
-    public final InetAddress fetchingNode;
-    public final InetAddress fetchFrom;
+    public final InetAddressAndPort initiator;
+    public final InetAddressAndPort fetchingNode;
+    public final InetAddressAndPort fetchFrom;
     public final Collection<Range<Token>> ranges;
     public final PreviewKind previewKind;
 
-    public AsymmetricSyncRequest(RepairJobDesc desc, InetAddress initiator, InetAddress fetchingNode, InetAddress fetchFrom, Collection<Range<Token>> ranges, PreviewKind previewKind)
+    public AsymmetricSyncRequest(RepairJobDesc desc, InetAddressAndPort initiator, InetAddressAndPort fetchingNode, InetAddressAndPort fetchFrom, Collection<Range<Token>> ranges, PreviewKind previewKind)
     {
         super(Type.ASYMMETRIC_SYNC_REQUEST, desc);
         this.initiator = initiator;
@@ -80,9 +81,9 @@ public class AsymmetricSyncRequest extends RepairMessage
         public void serialize(AsymmetricSyncRequest message, DataOutputPlus out, int version) throws IOException
         {
             RepairJobDesc.serializer.serialize(message.desc, out, version);
-            CompactEndpointSerializationHelper.serialize(message.initiator, out);
-            CompactEndpointSerializationHelper.serialize(message.fetchingNode, out);
-            CompactEndpointSerializationHelper.serialize(message.fetchFrom, out);
+            CompactEndpointSerializationHelper.instance.serialize(message.initiator, out, version);
+            CompactEndpointSerializationHelper.instance.serialize(message.fetchingNode, out, version);
+            CompactEndpointSerializationHelper.instance.serialize(message.fetchFrom, out, version);
             out.writeInt(message.ranges.size());
             for (Range<Token> range : message.ranges)
             {
@@ -95,9 +96,9 @@ public class AsymmetricSyncRequest extends RepairMessage
         public AsymmetricSyncRequest deserialize(DataInputPlus in, int version) throws IOException
         {
             RepairJobDesc desc = RepairJobDesc.serializer.deserialize(in, version);
-            InetAddress owner = CompactEndpointSerializationHelper.deserialize(in);
-            InetAddress src = CompactEndpointSerializationHelper.deserialize(in);
-            InetAddress dst = CompactEndpointSerializationHelper.deserialize(in);
+            InetAddressAndPort owner = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+            InetAddressAndPort src = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+            InetAddressAndPort dst = CompactEndpointSerializationHelper.instance.deserialize(in, version);
             int rangesCount = in.readInt();
             List<Range<Token>> ranges = new ArrayList<>(rangesCount);
             for (int i = 0; i < rangesCount; ++i)
@@ -109,7 +110,9 @@ public class AsymmetricSyncRequest extends RepairMessage
         public long serializedSize(AsymmetricSyncRequest message, int version)
         {
             long size = RepairJobDesc.serializer.serializedSize(message.desc, version);
-            size += 3 * CompactEndpointSerializationHelper.serializedSize(message.initiator);
+            size += CompactEndpointSerializationHelper.instance.serializedSize(message.initiator, version);
+            size += CompactEndpointSerializationHelper.instance.serializedSize(message.fetchingNode, version);
+            size += CompactEndpointSerializationHelper.instance.serializedSize(message.fetchFrom, version);
             size += TypeSizes.sizeof(message.ranges.size());
             for (Range<Token> range : message.ranges)
                 size += AbstractBounds.tokenSerializer.serializedSize(range, version);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java b/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java
index 6c28347..449748a 100644
--- a/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java
+++ b/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java
@@ -19,24 +19,22 @@
 package org.apache.cassandra.repair.messages;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.UUID;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.serializers.InetAddressSerializer;
-import org.apache.cassandra.serializers.TypeSerializer;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 public class FinalizePromise extends RepairMessage
 {
     public final UUID sessionID;
-    public final InetAddress participant;
+    public final InetAddressAndPort participant;
     public final boolean promised;
 
-    public FinalizePromise(UUID sessionID, InetAddress participant, boolean promised)
+    public FinalizePromise(UUID sessionID, InetAddressAndPort participant, boolean promised)
     {
         super(Type.FINALIZE_PROMISE, null);
         assert sessionID != null;
@@ -68,26 +66,24 @@ public class FinalizePromise extends RepairMessage
 
     public static MessageSerializer serializer = new MessageSerializer<FinalizePromise>()
     {
-        private TypeSerializer<InetAddress> inetSerializer = InetAddressSerializer.instance;
-
         public void serialize(FinalizePromise msg, DataOutputPlus out, int version) throws IOException
         {
             UUIDSerializer.serializer.serialize(msg.sessionID, out, version);
-            ByteBufferUtil.writeWithShortLength(inetSerializer.serialize(msg.participant), out);
+            CompactEndpointSerializationHelper.instance.serialize(msg.participant, out, version);
             out.writeBoolean(msg.promised);
         }
 
         public FinalizePromise deserialize(DataInputPlus in, int version) throws IOException
         {
             return new FinalizePromise(UUIDSerializer.serializer.deserialize(in, version),
-                                       inetSerializer.deserialize(ByteBufferUtil.readWithShortLength(in)),
+                                       CompactEndpointSerializationHelper.instance.deserialize(in, version),
                                        in.readBoolean());
         }
 
         public long serializedSize(FinalizePromise msg, int version)
         {
             long size = UUIDSerializer.serializer.serializedSize(msg.sessionID, version);
-            size += ByteBufferUtil.serializedSizeWithShortLength(inetSerializer.serialize(msg.participant));
+            size += CompactEndpointSerializationHelper.instance.serializedSize(msg.participant, version);
             size += TypeSizes.sizeof(msg.promised);
             return size;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java
index 57056ef..9aae256 100644
--- a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java
+++ b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.repair.messages;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.UUID;
@@ -29,18 +28,17 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.serializers.InetAddressSerializer;
-import org.apache.cassandra.serializers.TypeSerializer;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 public class PrepareConsistentRequest extends RepairMessage
 {
     public final UUID parentSession;
-    public final InetAddress coordinator;
-    public final Set<InetAddress> participants;
+    public final InetAddressAndPort coordinator;
+    public final Set<InetAddressAndPort> participants;
 
-    public PrepareConsistentRequest(UUID parentSession, InetAddress coordinator, Set<InetAddress> participants)
+    public PrepareConsistentRequest(UUID parentSession, InetAddressAndPort coordinator, Set<InetAddressAndPort> participants)
     {
         super(Type.CONSISTENT_REQUEST, null);
         assert parentSession != null;
@@ -82,28 +80,27 @@ public class PrepareConsistentRequest extends RepairMessage
 
     public static MessageSerializer serializer = new MessageSerializer<PrepareConsistentRequest>()
     {
-        private TypeSerializer<InetAddress> inetSerializer = InetAddressSerializer.instance;
 
         public void serialize(PrepareConsistentRequest request, DataOutputPlus out, int version) throws IOException
         {
             UUIDSerializer.serializer.serialize(request.parentSession, out, version);
-            ByteBufferUtil.writeWithShortLength(inetSerializer.serialize(request.coordinator), out);
+            CompactEndpointSerializationHelper.instance.serialize(request.coordinator, out, version);
             out.writeInt(request.participants.size());
-            for (InetAddress peer : request.participants)
+            for (InetAddressAndPort peer : request.participants)
             {
-                ByteBufferUtil.writeWithShortLength(inetSerializer.serialize(peer), out);
+                CompactEndpointSerializationHelper.instance.serialize(peer, out, version);
             }
         }
 
         public PrepareConsistentRequest deserialize(DataInputPlus in, int version) throws IOException
         {
             UUID sessionId = UUIDSerializer.serializer.deserialize(in, version);
-            InetAddress coordinator = inetSerializer.deserialize(ByteBufferUtil.readWithShortLength(in));
+            InetAddressAndPort coordinator = CompactEndpointSerializationHelper.instance.deserialize(in, version);
             int numPeers = in.readInt();
-            Set<InetAddress> peers = new HashSet<>(numPeers);
+            Set<InetAddressAndPort> peers = new HashSet<>(numPeers);
             for (int i = 0; i < numPeers; i++)
             {
-                InetAddress peer = inetSerializer.deserialize(ByteBufferUtil.readWithShortLength(in));
+                InetAddressAndPort peer = CompactEndpointSerializationHelper.instance.deserialize(in, version);
                 peers.add(peer);
             }
             return new PrepareConsistentRequest(sessionId, coordinator, peers);
@@ -112,11 +109,11 @@ public class PrepareConsistentRequest extends RepairMessage
         public long serializedSize(PrepareConsistentRequest request, int version)
         {
             long size = UUIDSerializer.serializer.serializedSize(request.parentSession, version);
-            size += ByteBufferUtil.serializedSizeWithShortLength(inetSerializer.serialize(request.coordinator));
+            size += CompactEndpointSerializationHelper.instance.serializedSize(request.coordinator, version);
             size += TypeSizes.sizeof(request.participants.size());
-            for (InetAddress peer : request.participants)
+            for (InetAddressAndPort peer : request.participants)
             {
-                size += ByteBufferUtil.serializedSizeWithShortLength(inetSerializer.serialize(peer));
+                size += CompactEndpointSerializationHelper.instance.serializedSize(peer, version);
             }
             return size;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java
index cf4410a..630f18e 100644
--- a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java
+++ b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java
@@ -19,24 +19,22 @@
 package org.apache.cassandra.repair.messages;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.UUID;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.serializers.InetAddressSerializer;
-import org.apache.cassandra.serializers.TypeSerializer;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 public class PrepareConsistentResponse extends RepairMessage
 {
     public final UUID parentSession;
-    public final InetAddress participant;
+    public final InetAddressAndPort participant;
     public final boolean success;
 
-    public PrepareConsistentResponse(UUID parentSession, InetAddress participant, boolean success)
+    public PrepareConsistentResponse(UUID parentSession, InetAddressAndPort participant, boolean success)
     {
         super(Type.CONSISTENT_RESPONSE, null);
         assert parentSession != null;
@@ -68,25 +66,24 @@ public class PrepareConsistentResponse extends RepairMessage
 
     public static MessageSerializer serializer = new MessageSerializer<PrepareConsistentResponse>()
     {
-        private TypeSerializer<InetAddress> inetSerializer = InetAddressSerializer.instance;
         public void serialize(PrepareConsistentResponse response, DataOutputPlus out, int version) throws IOException
         {
             UUIDSerializer.serializer.serialize(response.parentSession, out, version);
-            ByteBufferUtil.writeWithShortLength(inetSerializer.serialize(response.participant), out);
+            CompactEndpointSerializationHelper.instance.serialize(response.participant, out, version);
             out.writeBoolean(response.success);
         }
 
         public PrepareConsistentResponse deserialize(DataInputPlus in, int version) throws IOException
         {
             return new PrepareConsistentResponse(UUIDSerializer.serializer.deserialize(in, version),
-                                                 inetSerializer.deserialize(ByteBufferUtil.readWithShortLength(in)),
+                                                 CompactEndpointSerializationHelper.instance.deserialize(in, version),
                                                  in.readBoolean());
         }
 
         public long serializedSize(PrepareConsistentResponse response, int version)
         {
             long size = UUIDSerializer.serializer.serializedSize(response.parentSession, version);
-            size += ByteBufferUtil.serializedSizeWithShortLength(inetSerializer.serialize(response.participant));
+            size += CompactEndpointSerializationHelper.instance.serializedSize(response.participant, version);
             size += TypeSizes.sizeof(response.success);
             return size;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/messages/SyncComplete.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/messages/SyncComplete.java b/src/java/org/apache/cassandra/repair/messages/SyncComplete.java
index 7b68daf..1f1344d 100644
--- a/src/java/org/apache/cassandra/repair/messages/SyncComplete.java
+++ b/src/java/org/apache/cassandra/repair/messages/SyncComplete.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.repair.messages;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
@@ -26,6 +25,7 @@ import java.util.Objects;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.NodePair;
 import org.apache.cassandra.repair.RepairJobDesc;
 import org.apache.cassandra.streaming.SessionSummary;
@@ -53,7 +53,7 @@ public class SyncComplete extends RepairMessage
         this.summaries = summaries;
     }
 
-    public SyncComplete(RepairJobDesc desc, InetAddress endpoint1, InetAddress endpoint2, boolean success, List<SessionSummary> summaries)
+    public SyncComplete(RepairJobDesc desc, InetAddressAndPort endpoint1, InetAddressAndPort endpoint2, boolean success, List<SessionSummary> summaries)
     {
         super(Type.SYNC_COMPLETE, desc);
         this.summaries = summaries;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/repair/messages/SyncRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/messages/SyncRequest.java b/src/java/org/apache/cassandra/repair/messages/SyncRequest.java
index 01601e2..a0bf4e2 100644
--- a/src/java/org/apache/cassandra/repair/messages/SyncRequest.java
+++ b/src/java/org/apache/cassandra/repair/messages/SyncRequest.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.repair.messages;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -30,6 +29,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.RepairJobDesc;
@@ -45,14 +45,14 @@ public class SyncRequest extends RepairMessage
 {
     public static MessageSerializer serializer = new SyncRequestSerializer();
 
-    public final InetAddress initiator;
-    public final InetAddress src;
-    public final InetAddress dst;
+    public final InetAddressAndPort initiator;
+    public final InetAddressAndPort src;
+    public final InetAddressAndPort dst;
     public final Collection<Range<Token>> ranges;
     public final PreviewKind previewKind;
 
-    public SyncRequest(RepairJobDesc desc, InetAddress initiator, InetAddress src, InetAddress dst, Collection<Range<Token>> ranges, PreviewKind previewKind)
-    {
+   public SyncRequest(RepairJobDesc desc, InetAddressAndPort initiator, InetAddressAndPort src, InetAddressAndPort dst, Collection<Range<Token>> ranges, PreviewKind previewKind)
+   {
         super(Type.SYNC_REQUEST, desc);
         this.initiator = initiator;
         this.src = src;
@@ -87,9 +87,9 @@ public class SyncRequest extends RepairMessage
         public void serialize(SyncRequest message, DataOutputPlus out, int version) throws IOException
         {
             RepairJobDesc.serializer.serialize(message.desc, out, version);
-            CompactEndpointSerializationHelper.serialize(message.initiator, out);
-            CompactEndpointSerializationHelper.serialize(message.src, out);
-            CompactEndpointSerializationHelper.serialize(message.dst, out);
+            CompactEndpointSerializationHelper.instance.serialize(message.initiator, out, version);
+            CompactEndpointSerializationHelper.instance.serialize(message.src, out, version);
+            CompactEndpointSerializationHelper.instance.serialize(message.dst, out, version);
             out.writeInt(message.ranges.size());
             for (Range<Token> range : message.ranges)
             {
@@ -102,9 +102,9 @@ public class SyncRequest extends RepairMessage
         public SyncRequest deserialize(DataInputPlus in, int version) throws IOException
         {
             RepairJobDesc desc = RepairJobDesc.serializer.deserialize(in, version);
-            InetAddress owner = CompactEndpointSerializationHelper.deserialize(in);
-            InetAddress src = CompactEndpointSerializationHelper.deserialize(in);
-            InetAddress dst = CompactEndpointSerializationHelper.deserialize(in);
+            InetAddressAndPort owner = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+            InetAddressAndPort src = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+            InetAddressAndPort dst = CompactEndpointSerializationHelper.instance.deserialize(in, version);
             int rangesCount = in.readInt();
             List<Range<Token>> ranges = new ArrayList<>(rangesCount);
             for (int i = 0; i < rangesCount; ++i)
@@ -116,7 +116,7 @@ public class SyncRequest extends RepairMessage
         public long serializedSize(SyncRequest message, int version)
         {
             long size = RepairJobDesc.serializer.serializedSize(message.desc, version);
-            size += 3 * CompactEndpointSerializationHelper.serializedSize(message.initiator);
+            size += 3 * CompactEndpointSerializationHelper.instance.serializedSize(message.initiator, version);
             size += TypeSizes.sizeof(message.ranges.size());
             for (Range<Token> range : message.ranges)
                 size += AbstractBounds.tokenSerializer.serializedSize(range, version);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/schema/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/MigrationManager.java b/src/java/org/apache/cassandra/schema/MigrationManager.java
index ef19c25..c8881e5 100644
--- a/src/java/org/apache/cassandra/schema/MigrationManager.java
+++ b/src/java/org/apache/cassandra/schema/MigrationManager.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.schema;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.*;
 import java.lang.management.ManagementFactory;
@@ -39,6 +38,7 @@ import org.apache.cassandra.gms.*;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
@@ -59,10 +59,10 @@ public class MigrationManager
 
     private MigrationManager() {}
 
-    public static void scheduleSchemaPull(InetAddress endpoint, EndpointState state)
+    public static void scheduleSchemaPull(InetAddressAndPort endpoint, EndpointState state)
     {
         UUID schemaVersion = state.getSchemaVersion();
-        if (!endpoint.equals(FBUtilities.getBroadcastAddress()) && schemaVersion != null)
+        if (!endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) && schemaVersion != null)
             maybeScheduleSchemaPull(schemaVersion, endpoint);
     }
 
@@ -70,7 +70,7 @@ public class MigrationManager
      * If versions differ this node sends request with local migration list to the endpoint
      * and expecting to receive a list of migrations to apply locally.
      */
-    private static void maybeScheduleSchemaPull(final UUID theirVersion, final InetAddress endpoint)
+    private static void maybeScheduleSchemaPull(final UUID theirVersion, final InetAddressAndPort endpoint)
     {
         if (Schema.instance.getVersion() == null)
         {
@@ -130,7 +130,7 @@ public class MigrationManager
         }
     }
 
-    private static Future<?> submitMigrationTask(InetAddress endpoint)
+    private static Future<?> submitMigrationTask(InetAddressAndPort endpoint)
     {
         /*
          * Do not de-ref the future because that causes distributed deadlock (CASSANDRA-3832) because we are
@@ -139,7 +139,7 @@ public class MigrationManager
         return StageManager.getStage(Stage.MIGRATION).submit(new MigrationTask(endpoint));
     }
 
-    static boolean shouldPullSchemaFrom(InetAddress endpoint)
+    static boolean shouldPullSchemaFrom(InetAddressAndPort endpoint)
     {
         /*
          * Don't request schema from nodes with a differnt or unknonw major version (may have incompatible schema)
@@ -427,7 +427,7 @@ public class MigrationManager
             FBUtilities.waitOnFuture(announce(mutations));
     }
 
-    private static void pushSchemaMutation(InetAddress endpoint, Collection<Mutation> schema)
+    private static void pushSchemaMutation(InetAddressAndPort endpoint, Collection<Mutation> schema)
     {
         MessageOut<Collection<Mutation>> msg = new MessageOut<>(MessagingService.Verb.DEFINITIONS_UPDATE,
                                                                 schema,
@@ -446,10 +446,10 @@ public class MigrationManager
             }
         });
 
-        for (InetAddress endpoint : Gossiper.instance.getLiveMembers())
+        for (InetAddressAndPort endpoint : Gossiper.instance.getLiveMembers())
         {
             // only push schema to nodes with known and equal versions
-            if (!endpoint.equals(FBUtilities.getBroadcastAddress()) &&
+            if (!endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) &&
                     MessagingService.instance().knowsVersion(endpoint) &&
                     MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version)
                 pushSchemaMutation(endpoint, schema);
@@ -486,11 +486,11 @@ public class MigrationManager
 
         Schema.instance.clear();
 
-        Set<InetAddress> liveEndpoints = Gossiper.instance.getLiveMembers();
-        liveEndpoints.remove(FBUtilities.getBroadcastAddress());
+        Set<InetAddressAndPort> liveEndpoints = Gossiper.instance.getLiveMembers();
+        liveEndpoints.remove(FBUtilities.getBroadcastAddressAndPort());
 
         // force migration if there are nodes around
-        for (InetAddress node : liveEndpoints)
+        for (InetAddressAndPort node : liveEndpoints)
         {
             if (shouldPullSchemaFrom(node))
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/schema/MigrationTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/MigrationTask.java b/src/java/org/apache/cassandra/schema/MigrationTask.java
index 73e396d..6ff206a 100644
--- a/src/java/org/apache/cassandra/schema/MigrationTask.java
+++ b/src/java/org/apache/cassandra/schema/MigrationTask.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.schema;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.Set;
@@ -32,6 +31,7 @@ import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.SystemKeyspace.BootstrapState;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IAsyncCallback;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -46,9 +46,9 @@ final class MigrationTask extends WrappedRunnable
 
     private static final Set<BootstrapState> monitoringBootstrapStates = EnumSet.of(BootstrapState.NEEDS_BOOTSTRAP, BootstrapState.IN_PROGRESS);
 
-    private final InetAddress endpoint;
+    private final InetAddressAndPort endpoint;
 
-    MigrationTask(InetAddress endpoint)
+    MigrationTask(InetAddressAndPort endpoint)
     {
         this.endpoint = endpoint;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
index 7ff0b9b..e06131e 100644
--- a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
+++ b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ThreadLocalRandom;
@@ -38,6 +37,7 @@ import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
@@ -60,12 +60,12 @@ public abstract class AbstractReadExecutor
     private static final Logger logger = LoggerFactory.getLogger(AbstractReadExecutor.class);
 
     protected final ReadCommand command;
-    protected final List<InetAddress> targetReplicas;
+    protected final List<InetAddressAndPort> targetReplicas;
     protected final ReadCallback handler;
     protected final TraceState traceState;
     protected final ColumnFamilyStore cfs;
 
-    AbstractReadExecutor(Keyspace keyspace, ColumnFamilyStore cfs, ReadCommand command, ConsistencyLevel consistencyLevel, List<InetAddress> targetReplicas, long queryStartNanoTime)
+    AbstractReadExecutor(Keyspace keyspace, ColumnFamilyStore cfs, ReadCommand command, ConsistencyLevel consistencyLevel, List<InetAddressAndPort> targetReplicas, long queryStartNanoTime)
     {
         this.command = command;
         this.targetReplicas = targetReplicas;
@@ -78,27 +78,27 @@ public abstract class AbstractReadExecutor
         // TODO: we need this when talking with pre-3.0 nodes. So if we preserve the digest format moving forward, we can get rid of this once
         // we stop being compatible with pre-3.0 nodes.
         int digestVersion = MessagingService.current_version;
-        for (InetAddress replica : targetReplicas)
+        for (InetAddressAndPort replica : targetReplicas)
             digestVersion = Math.min(digestVersion, MessagingService.instance().getVersion(replica));
         command.setDigestVersion(digestVersion);
     }
 
-    protected void makeDataRequests(Iterable<InetAddress> endpoints)
+    protected void makeDataRequests(Iterable<InetAddressAndPort> endpoints)
     {
         makeRequests(command, endpoints);
 
     }
 
-    protected void makeDigestRequests(Iterable<InetAddress> endpoints)
+    protected void makeDigestRequests(Iterable<InetAddressAndPort> endpoints)
     {
         makeRequests(command.copyAsDigestQuery(), endpoints);
     }
 
-    private void makeRequests(ReadCommand readCommand, Iterable<InetAddress> endpoints)
+    private void makeRequests(ReadCommand readCommand, Iterable<InetAddressAndPort> endpoints)
     {
         boolean hasLocalEndpoint = false;
 
-        for (InetAddress endpoint : endpoints)
+        for (InetAddressAndPort endpoint : endpoints)
         {
             if (StorageProxy.canDoLocalRequest(endpoint))
             {
@@ -132,7 +132,7 @@ public abstract class AbstractReadExecutor
      *
      * @return target replicas + the extra replica, *IF* we speculated.
      */
-    public abstract Collection<InetAddress> getContactedReplicas();
+    public abstract Collection<InetAddressAndPort> getContactedReplicas();
 
     /**
      * send the initial set of requests
@@ -184,12 +184,12 @@ public abstract class AbstractReadExecutor
     public static AbstractReadExecutor getReadExecutor(SinglePartitionReadCommand command, ConsistencyLevel consistencyLevel, long queryStartNanoTime) throws UnavailableException
     {
         Keyspace keyspace = Keyspace.open(command.metadata().keyspace);
-        List<InetAddress> allReplicas = StorageProxy.getLiveSortedEndpoints(keyspace, command.partitionKey());
+        List<InetAddressAndPort> allReplicas = StorageProxy.getLiveSortedEndpoints(keyspace, command.partitionKey());
         // 11980: Excluding EACH_QUORUM reads from potential RR, so that we do not miscount DC responses
         ReadRepairDecision repairDecision = consistencyLevel == ConsistencyLevel.EACH_QUORUM
                                             ? ReadRepairDecision.NONE
                                             : newReadRepairDecision(command.metadata());
-        List<InetAddress> targetReplicas = consistencyLevel.filterForQuery(keyspace, allReplicas, repairDecision);
+        List<InetAddressAndPort> targetReplicas = consistencyLevel.filterForQuery(keyspace, allReplicas, repairDecision);
 
         // Throw UAE early if we don't have enough replicas.
         consistencyLevel.assureSufficientLiveNodes(keyspace, targetReplicas);
@@ -223,12 +223,12 @@ public abstract class AbstractReadExecutor
         }
 
         // RRD.NONE or RRD.DC_LOCAL w/ multiple DCs.
-        InetAddress extraReplica = allReplicas.get(targetReplicas.size());
+        InetAddressAndPort extraReplica = allReplicas.get(targetReplicas.size());
         // With repair decision DC_LOCAL all replicas/target replicas may be in different order, so
         // we might have to find a replacement that's not already in targetReplicas.
         if (repairDecision == ReadRepairDecision.DC_LOCAL && targetReplicas.contains(extraReplica))
         {
-            for (InetAddress address : allReplicas)
+            for (InetAddressAndPort address : allReplicas)
             {
                 if (!targetReplicas.contains(address))
                 {
@@ -269,7 +269,7 @@ public abstract class AbstractReadExecutor
          */
         private final boolean logFailedSpeculation;
 
-        public NeverSpeculatingReadExecutor(Keyspace keyspace, ColumnFamilyStore cfs, ReadCommand command, ConsistencyLevel consistencyLevel, List<InetAddress> targetReplicas, long queryStartNanoTime, boolean logFailedSpeculation)
+        public NeverSpeculatingReadExecutor(Keyspace keyspace, ColumnFamilyStore cfs, ReadCommand command, ConsistencyLevel consistencyLevel, List<InetAddressAndPort> targetReplicas, long queryStartNanoTime, boolean logFailedSpeculation)
         {
             super(keyspace, cfs, command, consistencyLevel, targetReplicas, queryStartNanoTime);
             this.logFailedSpeculation = logFailedSpeculation;
@@ -290,7 +290,7 @@ public abstract class AbstractReadExecutor
             }
         }
 
-        public Collection<InetAddress> getContactedReplicas()
+        public Collection<InetAddressAndPort> getContactedReplicas()
         {
             return targetReplicas;
         }
@@ -304,7 +304,7 @@ public abstract class AbstractReadExecutor
                                        ColumnFamilyStore cfs,
                                        ReadCommand command,
                                        ConsistencyLevel consistencyLevel,
-                                       List<InetAddress> targetReplicas,
+                                       List<InetAddressAndPort> targetReplicas,
                                        long queryStartNanoTime)
         {
             super(keyspace, cfs, command, consistencyLevel, targetReplicas, queryStartNanoTime);
@@ -314,7 +314,7 @@ public abstract class AbstractReadExecutor
         {
             // if CL + RR result in covering all replicas, getReadExecutor forces AlwaysSpeculating.  So we know
             // that the last replica in our list is "extra."
-            List<InetAddress> initialReplicas = targetReplicas.subList(0, targetReplicas.size() - 1);
+            List<InetAddressAndPort> initialReplicas = targetReplicas.subList(0, targetReplicas.size() - 1);
 
             if (handler.blockfor < initialReplicas.size())
             {
@@ -347,7 +347,7 @@ public abstract class AbstractReadExecutor
                 if (handler.resolver.isDataPresent())
                     retryCommand = command.copyAsDigestQuery();
 
-                InetAddress extraReplica = Iterables.getLast(targetReplicas);
+                InetAddressAndPort extraReplica = Iterables.getLast(targetReplicas);
                 if (traceState != null)
                     traceState.trace("speculating read retry on {}", extraReplica);
                 logger.trace("speculating read retry on {}", extraReplica);
@@ -355,7 +355,7 @@ public abstract class AbstractReadExecutor
             }
         }
 
-        public Collection<InetAddress> getContactedReplicas()
+        public Collection<InetAddressAndPort> getContactedReplicas()
         {
             return speculated
                  ? targetReplicas
@@ -378,7 +378,7 @@ public abstract class AbstractReadExecutor
                                              ColumnFamilyStore cfs,
                                              ReadCommand command,
                                              ConsistencyLevel consistencyLevel,
-                                             List<InetAddress> targetReplicas,
+                                             List<InetAddressAndPort> targetReplicas,
                                              long queryStartNanoTime)
         {
             super(keyspace, cfs, command, consistencyLevel, targetReplicas, queryStartNanoTime);
@@ -389,7 +389,7 @@ public abstract class AbstractReadExecutor
             // no-op
         }
 
-        public Collection<InetAddress> getContactedReplicas()
+        public Collection<InetAddressAndPort> getContactedReplicas()
         {
             return targetReplicas;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
index b5eaadb..9d800a0 100644
--- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.service;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -35,6 +34,7 @@ import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
@@ -47,15 +47,15 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
     private AtomicInteger responsesAndExpirations;
     private final SimpleCondition condition = new SimpleCondition();
     protected final Keyspace keyspace;
-    protected final Collection<InetAddress> naturalEndpoints;
+    protected final Collection<InetAddressAndPort> naturalEndpoints;
     public final ConsistencyLevel consistencyLevel;
     protected final Runnable callback;
-    protected final Collection<InetAddress> pendingEndpoints;
+    protected final Collection<InetAddressAndPort> pendingEndpoints;
     protected final WriteType writeType;
     private static final AtomicIntegerFieldUpdater<AbstractWriteResponseHandler> failuresUpdater
     = AtomicIntegerFieldUpdater.newUpdater(AbstractWriteResponseHandler.class, "failures");
     private volatile int failures = 0;
-    private final Map<InetAddress, RequestFailureReason> failureReasonByEndpoint;
+    private final Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint;
     private final long queryStartNanoTime;
     private volatile boolean supportsBackPressure = true;
 
@@ -72,8 +72,8 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
      * @param queryStartNanoTime
      */
     protected AbstractWriteResponseHandler(Keyspace keyspace,
-                                           Collection<InetAddress> naturalEndpoints,
-                                           Collection<InetAddress> pendingEndpoints,
+                                           Collection<InetAddressAndPort> naturalEndpoints,
+                                           Collection<InetAddressAndPort> pendingEndpoints,
                                            ConsistencyLevel consistencyLevel,
                                            Runnable callback,
                                            WriteType writeType,
@@ -208,7 +208,7 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
     /**
      * @return true if the message counts towards the totalBlockFor() threshold
      */
-    protected boolean waitingFor(InetAddress from)
+    protected boolean waitingFor(InetAddressAndPort from)
     {
         return true;
     }
@@ -236,7 +236,7 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
     }
 
     @Override
-    public void onFailure(InetAddress from, RequestFailureReason failureReason)
+    public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
     {
         logger.trace("Got failure from {}", from);
 


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


[14/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/ForwardToSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/ForwardToSerializer.java b/src/java/org/apache/cassandra/net/ForwardToSerializer.java
new file mode 100644
index 0000000..c4e8843
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/ForwardToSerializer.java
@@ -0,0 +1,86 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+public class ForwardToSerializer implements IVersionedSerializer<ForwardToContainer>
+{
+    public static ForwardToSerializer instance = new ForwardToSerializer();
+
+    private ForwardToSerializer() {}
+
+    public void serialize(ForwardToContainer forwardToContainer, DataOutputPlus out, int version) throws IOException
+    {
+        out.writeInt(forwardToContainer.targets.size());
+        Iterator<InetAddressAndPort> iter = forwardToContainer.targets.iterator();
+        for (int ii = 0; ii < forwardToContainer.messageIds.length; ii++)
+        {
+            CompactEndpointSerializationHelper.instance.serialize(iter.next(), out, version);
+            out.writeInt(forwardToContainer.messageIds[ii]);
+        }
+    }
+
+    public ForwardToContainer deserialize(DataInputPlus in, int version) throws IOException
+    {
+        int[] ids = new int[in.readInt()];
+        List<InetAddressAndPort> hosts = new ArrayList<>(ids.length);
+        for (int ii = 0; ii < ids.length; ii++)
+        {
+           hosts.add(CompactEndpointSerializationHelper.instance.deserialize(in, version));
+           ids[ii] = in.readInt();
+        }
+        return new ForwardToContainer(hosts, ids);
+    }
+
+    public long serializedSize(ForwardToContainer forwardToContainer, int version)
+    {
+        //Number of forward addresses, 4 bytes per for each id
+        long size = 4 +
+                    (4 * forwardToContainer.targets.size());
+        //Depending on ipv6 or ipv4 the address size is different.
+        for (InetAddressAndPort forwardTo : forwardToContainer.targets)
+        {
+            size += CompactEndpointSerializationHelper.instance.serializedSize(forwardTo, version);
+        }
+
+        return size;
+    }
+
+    public static ForwardToContainer fromBytes(byte[] bytes, int version)
+    {
+        try (DataInputBuffer input = new DataInputBuffer(bytes))
+        {
+            return instance.deserialize(input, version);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/IAsyncCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IAsyncCallback.java b/src/java/org/apache/cassandra/net/IAsyncCallback.java
index 7835079..251d263 100644
--- a/src/java/org/apache/cassandra/net/IAsyncCallback.java
+++ b/src/java/org/apache/cassandra/net/IAsyncCallback.java
@@ -17,11 +17,10 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
-
 import com.google.common.base.Predicate;
 
 import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * implementors of IAsyncCallback need to make sure that any public methods
@@ -31,9 +30,9 @@ import org.apache.cassandra.gms.FailureDetector;
  */
 public interface IAsyncCallback<T>
 {
-    Predicate<InetAddress> isAlive = new Predicate<InetAddress>()
+    Predicate<InetAddressAndPort> isAlive = new Predicate<InetAddressAndPort>()
     {
-        public boolean apply(InetAddress endpoint)
+        public boolean apply(InetAddressAndPort endpoint)
         {
             return FailureDetector.instance.isAlive(endpoint);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/IAsyncCallbackWithFailure.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IAsyncCallbackWithFailure.java b/src/java/org/apache/cassandra/net/IAsyncCallbackWithFailure.java
index 1cd27b6..2b91f20 100644
--- a/src/java/org/apache/cassandra/net/IAsyncCallbackWithFailure.java
+++ b/src/java/org/apache/cassandra/net/IAsyncCallbackWithFailure.java
@@ -17,9 +17,8 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
-
 import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public interface IAsyncCallbackWithFailure<T> extends IAsyncCallback<T>
 {
@@ -27,5 +26,5 @@ public interface IAsyncCallbackWithFailure<T> extends IAsyncCallback<T>
     /**
      * Called when there is an exception on the remote node or timeout happens
      */
-    void onFailure(InetAddress from, RequestFailureReason failureReason);
+    void onFailure(InetAddressAndPort from, RequestFailureReason failureReason);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/IMessageSink.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IMessageSink.java b/src/java/org/apache/cassandra/net/IMessageSink.java
index 5150901..090d2c2 100644
--- a/src/java/org/apache/cassandra/net/IMessageSink.java
+++ b/src/java/org/apache/cassandra/net/IMessageSink.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public interface IMessageSink
 {
@@ -26,7 +26,7 @@ public interface IMessageSink
      *
      * @return true if the message is allowed, false if it should be dropped
      */
-    boolean allowOutgoingMessage(MessageOut message, int id, InetAddress to);
+    boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to);
 
     /**
      * Allow or drop an incoming message

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
index c7fc991..6e132a8 100644
--- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
+++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.net;
 import java.io.IOException;
 import java.util.EnumSet;
 
+import com.google.common.primitives.Shorts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -28,6 +29,7 @@ import org.apache.cassandra.db.monitoring.ApproximateTime;
 import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.index.IndexNotAvailableException;
+import org.apache.cassandra.io.DummyByteVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 
 public class MessageDeliveryTask implements Runnable
@@ -96,19 +98,11 @@ public class MessageDeliveryTask implements Runnable
         if (message.doCallbackOnFailure())
         {
             MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
-                                                .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE);
+                                                .withParameter(ParameterType.FAILURE_RESPONSE, MessagingService.ONE_BYTE);
 
             if (t instanceof TombstoneOverwhelmingException)
             {
-                try (DataOutputBuffer out = new DataOutputBuffer())
-                {
-                    out.writeShort(RequestFailureReason.READ_TOO_MANY_TOMBSTONES.code);
-                    response = response.withParameter(MessagingService.FAILURE_REASON_PARAM, out.getData());
-                }
-                catch (IOException ex)
-                {
-                    throw new RuntimeException(ex);
-                }
+                response = response.withParameter(ParameterType.FAILURE_REASON, Shorts.checkedCast(RequestFailureReason.READ_TOO_MANY_TOMBSTONES.code));
             }
 
             MessagingService.instance().sendReply(response, id, message.from);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 d520fa9..a426ef0 100644
--- a/src/java/org/apache/cassandra/net/MessageIn.java
+++ b/src/java/org/apache/cassandra/net/MessageIn.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.net;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.Map;
 
@@ -31,8 +30,8 @@ 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.locator.InetAddressAndPort;
 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.
@@ -41,16 +40,16 @@ import org.apache.cassandra.net.MessagingService.Verb;
  */
 public class MessageIn<T>
 {
-    public final InetAddress from;
+    public final InetAddressAndPort from;
     public final T payload;
-    public final Map<String, byte[]> parameters;
-    public final Verb verb;
+    public final Map<ParameterType, Object> parameters;
+    public final MessagingService.Verb verb;
     public final int version;
     public final long constructionTime;
 
-    private MessageIn(InetAddress from,
+    private MessageIn(InetAddressAndPort from,
                       T payload,
-                      Map<String, byte[]> parameters,
+                      Map<ParameterType, Object> parameters,
                       Verb verb,
                       int version,
                       long constructionTime)
@@ -63,9 +62,9 @@ public class MessageIn<T>
         this.constructionTime = constructionTime;
     }
 
-    public static <T> MessageIn<T> create(InetAddress from,
+    public static <T> MessageIn<T> create(InetAddressAndPort from,
                                           T payload,
-                                          Map<String, byte[]> parameters,
+                                          Map<ParameterType, Object> parameters,
                                           Verb verb,
                                           int version,
                                           long constructionTime)
@@ -73,9 +72,9 @@ public class MessageIn<T>
         return new MessageIn<>(from, payload, parameters, verb, version, constructionTime);
     }
 
-    public static <T> MessageIn<T> create(InetAddress from,
+    public static <T> MessageIn<T> create(InetAddressAndPort from,
                                           T payload,
-                                          Map<String, byte[]> parameters,
+                                          Map<ParameterType, Object> parameters,
                                           MessagingService.Verb verb,
                                           int version)
     {
@@ -89,37 +88,46 @@ public class MessageIn<T>
 
     public static <T2> MessageIn<T2> read(DataInputPlus in, int version, int id, long constructionTime) throws IOException
     {
-        InetAddress from = CompactEndpointSerializationHelper.deserialize(in);
+        InetAddressAndPort from = CompactEndpointSerializationHelper.instance.deserialize(in, version);
 
         MessagingService.Verb verb = MessagingService.Verb.fromId(in.readInt());
-        Map<String, byte[]> parameters = readParameters(in);
+        Map<ParameterType, Object> parameters = readParameters(in, version);
         int payloadSize = in.readInt();
         return read(in, version, id, constructionTime, from, payloadSize, verb, parameters);
     }
 
-    public static Map<String, byte[]> readParameters(DataInputPlus in) throws IOException
+    public static Map<ParameterType, Object> readParameters(DataInputPlus in, int version) throws IOException
     {
         int parameterCount = in.readInt();
+        Map<ParameterType, Object> parameters;
         if (parameterCount == 0)
         {
             return Collections.emptyMap();
         }
         else
         {
-            ImmutableMap.Builder<String, byte[]> builder = ImmutableMap.builder();
+            ImmutableMap.Builder<ParameterType, Object> builder = ImmutableMap.builder();
             for (int i = 0; i < parameterCount; i++)
             {
                 String key = in.readUTF();
-                byte[] value = new byte[in.readInt()];
-                in.readFully(value);
-                builder.put(key, value);
+                ParameterType type = ParameterType.byName.get(key);
+                if (type != null)
+                {
+                    byte[] value = new byte[in.readInt()];
+                    in.readFully(value);
+                    builder.put(type, type.serializer.deserialize(new DataInputBuffer(value), version));
+                }
+                else
+                {
+                    in.skipBytes(in.readInt());
+                }
             }
             return builder.build();
         }
     }
 
     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
+                                          InetAddressAndPort from, int payloadSize, Verb verb, Map<ParameterType, Object> parameters) throws IOException
     {
         IVersionedSerializer<T2> serializer = (IVersionedSerializer<T2>) MessagingService.verbSerializers.get(verb);
         if (serializer instanceof MessagingService.CallbackDeterminedSerializer)
@@ -140,7 +148,7 @@ public class MessageIn<T>
         return MessageIn.create(from, payload, parameters, verb, version, constructionTime);
     }
 
-    public static long deriveConstructionTime(InetAddress from, int messageTimestamp, long currentTime)
+    public static long deriveConstructionTime(InetAddressAndPort 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)
@@ -182,36 +190,18 @@ public class MessageIn<T>
 
     public boolean doCallbackOnFailure()
     {
-        return parameters.containsKey(MessagingService.FAILURE_CALLBACK_PARAM);
+        return parameters.containsKey(ParameterType.FAILURE_CALLBACK);
     }
 
     public boolean isFailureResponse()
     {
-        return parameters.containsKey(MessagingService.FAILURE_RESPONSE_PARAM);
-    }
-
-    public boolean containsFailureReason()
-    {
-        return parameters.containsKey(MessagingService.FAILURE_REASON_PARAM);
+        return parameters.containsKey(ParameterType.FAILURE_RESPONSE);
     }
 
     public RequestFailureReason getFailureReason()
     {
-        if (containsFailureReason())
-        {
-            try (DataInputBuffer in = new DataInputBuffer(parameters.get(MessagingService.FAILURE_REASON_PARAM)))
-            {
-                return RequestFailureReason.fromCode(in.readUnsignedShort());
-            }
-            catch (IOException ex)
-            {
-                throw new RuntimeException(ex);
-            }
-        }
-        else
-        {
-            return RequestFailureReason.UNKNOWN;
-        }
+        Short code = (Short)parameters.get(ParameterType.FAILURE_REASON);
+        return code != null ? RequestFailureReason.fromCode(code) : RequestFailureReason.UNKNOWN;
     }
 
     public long getTimeout()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 379aff5..7d3c0af 100644
--- a/src/java/org/apache/cassandra/net/MessageOut.java
+++ b/src/java/org/apache/cassandra/net/MessageOut.java
@@ -19,17 +19,18 @@
 package org.apache.cassandra.net;
 
 import java.io.IOException;
-import java.net.InetAddress;
-import java.util.Collections;
-import java.util.Map;
+import java.util.ArrayList;
+import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.primitives.Ints;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
@@ -80,12 +81,20 @@ import static org.apache.cassandra.tracing.Tracing.isTracing;
 public class MessageOut<T>
 {
     private static final int SERIALIZED_SIZE_VERSION_UNDEFINED = -1;
+    //Parameters are stored in an object array as tuples of size two
+    public static final int PARAMETER_TUPLE_SIZE = 2;
+    //Offset in a parameter tuple containing the type of the parameter
+    public static final int PARAMETER_TUPLE_TYPE_OFFSET = 0;
+    //Offset in a parameter tuple containing the actual parameter represented as a POJO
+    public static final int PARAMETER_TUPLE_PARAMETER_OFFSET = 1;
 
-    public final InetAddress from;
+    public final InetAddressAndPort from;
     public final MessagingService.Verb verb;
     public final T payload;
     public final IVersionedSerializer<T> serializer;
-    public final Map<String, byte[]> parameters;
+    //A list of tuples, first object is the ParameterType enum value,
+    //the second object is the POJO to serialize
+    public final List<Object> parameters;
 
     /**
      * Memoization of the serialized size of the just the payload.
@@ -115,16 +124,16 @@ public class MessageOut<T>
              serializer,
              isTracing()
                  ? Tracing.instance.getTraceHeaders()
-                 : Collections.<String, byte[]>emptyMap());
+                 : ImmutableList.of());
     }
 
-    private MessageOut(MessagingService.Verb verb, T payload, IVersionedSerializer<T> serializer, Map<String, byte[]> parameters)
+    private MessageOut(MessagingService.Verb verb, T payload, IVersionedSerializer<T> serializer, List<Object> parameters)
     {
-        this(FBUtilities.getBroadcastAddress(), verb, payload, serializer, parameters);
+        this(FBUtilities.getBroadcastAddressAndPort(), verb, payload, serializer, parameters);
     }
 
     @VisibleForTesting
-    public MessageOut(InetAddress from, MessagingService.Verb verb, T payload, IVersionedSerializer<T> serializer, Map<String, byte[]> parameters)
+    public MessageOut(InetAddressAndPort from, MessagingService.Verb verb, T payload, IVersionedSerializer<T> serializer, List<Object> parameters)
     {
         this.from = from;
         this.verb = verb;
@@ -133,11 +142,13 @@ public class MessageOut<T>
         this.parameters = parameters;
     }
 
-    public MessageOut<T> withParameter(String key, byte[] value)
+    public <VT> MessageOut<T> withParameter(ParameterType type, VT value)
     {
-        ImmutableMap.Builder<String, byte[]> builder = ImmutableMap.builder();
-        builder.putAll(parameters).put(key, value);
-        return new MessageOut<T>(verb, payload, serializer, builder.build());
+        List<Object> newParameters = new ArrayList<>(parameters.size() + 2);
+        newParameters.addAll(parameters);
+        newParameters.add(type);
+        newParameters.add(value);
+        return new MessageOut<T>(verb, payload, serializer, newParameters);
     }
 
     public Stage getStage()
@@ -159,15 +170,19 @@ public class MessageOut<T>
 
     public void serialize(DataOutputPlus out, int version) throws IOException
     {
-        CompactEndpointSerializationHelper.serialize(from, out);
+        CompactEndpointSerializationHelper.instance.serialize(from, out, version);
 
         out.writeInt(verb.getId());
-        out.writeInt(parameters.size());
-        for (Map.Entry<String, byte[]> entry : parameters.entrySet())
+        assert parameters.size() % PARAMETER_TUPLE_SIZE == 0;
+        out.writeInt(parameters.size() / PARAMETER_TUPLE_SIZE);
+        for (int ii = 0; ii < parameters.size(); ii += PARAMETER_TUPLE_SIZE)
         {
-            out.writeUTF(entry.getKey());
-            out.writeInt(entry.getValue().length);
-            out.write(entry.getValue());
+            ParameterType type = (ParameterType)parameters.get(ii + PARAMETER_TUPLE_TYPE_OFFSET);
+            out.writeUTF(type.key);
+            IVersionedSerializer serializer = type.serializer;
+            Object parameter = parameters.get(ii + PARAMETER_TUPLE_PARAMETER_OFFSET);
+            out.writeInt(Ints.checkedCast(serializer.serializedSize(parameter, version)));
+            serializer.serialize(parameter, out, version);
         }
 
         if (payload != null)
@@ -187,15 +202,19 @@ public class MessageOut<T>
 
     private Pair<Long, Long> calculateSerializedSize(int version)
     {
-        long size = CompactEndpointSerializationHelper.serializedSize(from);
+        long size = 0;
+        size += CompactEndpointSerializationHelper.instance.serializedSize(from, version);
 
         size += TypeSizes.sizeof(verb.getId());
         size += TypeSizes.sizeof(parameters.size());
-        for (Map.Entry<String, byte[]> entry : parameters.entrySet())
+        for (int ii = 0; ii < parameters.size(); ii += PARAMETER_TUPLE_SIZE)
         {
-            size += TypeSizes.sizeof(entry.getKey());
-            size += TypeSizes.sizeof(entry.getValue().length);
-            size += entry.getValue().length;
+            ParameterType type = (ParameterType)parameters.get(ii + PARAMETER_TUPLE_TYPE_OFFSET);
+            size += TypeSizes.sizeof(type.key());
+            size += 4;//length prefix
+            IVersionedSerializer serializer = type.serializer;
+            Object parameter = parameters.get(ii + PARAMETER_TUPLE_PARAMETER_OFFSET);
+            size += serializer.serializedSize(parameter, version);
         }
 
         long payloadSize = payload == null ? 0 : serializer.serializedSize(payload, version);
@@ -237,4 +256,16 @@ public class MessageOut<T>
 
         return sizes.left.intValue();
     }
+
+    public Object getParameter(ParameterType type)
+    {
+        for (int ii = 0; ii < parameters.size(); ii += PARAMETER_TUPLE_SIZE)
+        {
+            if (((ParameterType)parameters.get(ii + PARAMETER_TUPLE_TYPE_OFFSET)).equals(type))
+            {
+                return parameters.get(ii + PARAMETER_TUPLE_PARAMETER_OFFSET);
+            }
+        }
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 4e6fe1c..9f00d27 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -20,8 +20,6 @@ package org.apache.cassandra.net;
 import java.io.IOError;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -37,6 +35,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
@@ -91,6 +90,7 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.ILatencySubscriber;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.metrics.ConnectionMetrics;
 import org.apache.cassandra.metrics.DroppedMessageMetrics;
@@ -127,10 +127,7 @@ public final class MessagingService implements MessagingServiceMBean
     public static final int VERSION_40 = 12;
     public static final int current_version = VERSION_40;
 
-    public static final String FAILURE_CALLBACK_PARAM = "CAL_BAC";
     public static final byte[] ONE_BYTE = new byte[1];
-    public static final String FAILURE_RESPONSE_PARAM = "FAIL";
-    public static final String FAILURE_REASON_PARAM = "FAIL_REASON";
 
     /**
      * we preface every message with this number so the recipient can validate the sender is sane
@@ -447,7 +444,7 @@ public final class MessagingService implements MessagingServiceMBean
     private final Map<Verb, IVerbHandler> verbHandlers;
 
     @VisibleForTesting
-    public final ConcurrentMap<InetAddress, OutboundMessagingPool> channelManagers = new NonBlockingHashMap<>();
+    public final ConcurrentMap<InetAddressAndPort, OutboundMessagingPool> channelManagers = new NonBlockingHashMap<>();
     final List<ServerChannel> serverChannels = Lists.newArrayList();
 
     private static final Logger logger = LoggerFactory.getLogger(MessagingService.class);
@@ -506,7 +503,7 @@ public final class MessagingService implements MessagingServiceMBean
     private final List<ILatencySubscriber> subscribers = new ArrayList<ILatencySubscriber>();
 
     // protocol versions of the other nodes in the cluster
-    private final ConcurrentMap<InetAddress, Integer> versions = new NonBlockingHashMap<InetAddress, Integer>();
+    private final ConcurrentMap<InetAddressAndPort, Integer> versions = new NonBlockingHashMap<>();
 
     // message sinks are a testing hook
     private final Set<IMessageSink> messageSinks = new CopyOnWriteArraySet<>();
@@ -629,7 +626,7 @@ public final class MessagingService implements MessagingServiceMBean
      * @param callback The message callback.
      * @param message The actual message.
      */
-    public void updateBackPressureOnSend(InetAddress host, IAsyncCallback callback, MessageOut<?> message)
+    public void updateBackPressureOnSend(InetAddressAndPort host, IAsyncCallback callback, MessageOut<?> message)
     {
         if (DatabaseDescriptor.backPressureEnabled() && callback.supportsBackPressure())
         {
@@ -646,7 +643,7 @@ public final class MessagingService implements MessagingServiceMBean
      * @param callback The message callback.
      * @param timeout True if updated following a timeout, false otherwise.
      */
-    public void updateBackPressureOnReceive(InetAddress host, IAsyncCallback callback, boolean timeout)
+    public void updateBackPressureOnReceive(InetAddressAndPort host, IAsyncCallback callback, boolean timeout)
     {
         if (DatabaseDescriptor.backPressureEnabled() && callback.supportsBackPressure())
         {
@@ -669,14 +666,14 @@ public final class MessagingService implements MessagingServiceMBean
      * @param hosts The hosts to apply back-pressure to.
      * @param timeoutInNanos The max back-pressure timeout.
      */
-    public void applyBackPressure(Iterable<InetAddress> hosts, long timeoutInNanos)
+    public void applyBackPressure(Iterable<InetAddressAndPort> hosts, long timeoutInNanos)
     {
         if (DatabaseDescriptor.backPressureEnabled())
         {
             Set<BackPressureState> states = new HashSet<BackPressureState>();
-            for (InetAddress host : hosts)
+            for (InetAddressAndPort host : hosts)
             {
-                if (host.equals(FBUtilities.getBroadcastAddress()))
+                if (host.equals(FBUtilities.getBroadcastAddressAndPort()))
                     continue;
                 OutboundMessagingPool pool = getMessagingConnection(host);
                 if (pool != null)
@@ -686,13 +683,13 @@ public final class MessagingService implements MessagingServiceMBean
         }
     }
 
-    BackPressureState getBackPressureState(InetAddress host)
+    BackPressureState getBackPressureState(InetAddressAndPort host)
     {
         OutboundMessagingPool messagingConnection = getMessagingConnection(host);
         return messagingConnection != null ? messagingConnection.getBackPressureState() : null;
     }
 
-    void markTimeout(InetAddress addr)
+    void markTimeout(InetAddressAndPort addr)
     {
         OutboundMessagingPool conn = channelManagers.get(addr);
         if (conn != null)
@@ -706,13 +703,13 @@ public final class MessagingService implements MessagingServiceMBean
      * @param address the host that replied to the message
      * @param latency
      */
-    public void maybeAddLatency(IAsyncCallback cb, InetAddress address, long latency)
+    public void maybeAddLatency(IAsyncCallback cb, InetAddressAndPort address, long latency)
     {
         if (cb.isLatencyForSnitch())
             addLatency(address, latency);
     }
 
-    public void addLatency(InetAddress address, long latency)
+    public void addLatency(InetAddressAndPort address, long latency)
     {
         for (ILatencySubscriber subscriber : subscribers)
             subscriber.receiveTiming(address, latency);
@@ -721,7 +718,7 @@ public final class MessagingService implements MessagingServiceMBean
     /**
      * called from gossiper when it notices a node is not responding.
      */
-    public void convict(InetAddress ep)
+    public void convict(InetAddressAndPort ep)
     {
         logger.trace("Resetting pool for {}", ep);
         reset(ep);
@@ -735,24 +732,24 @@ public final class MessagingService implements MessagingServiceMBean
     public void listen(ServerEncryptionOptions serverEncryptionOptions)
     {
         callbacks.reset(); // hack to allow tests to stop/restart MS
-        listen(FBUtilities.getLocalAddress(), serverEncryptionOptions);
+        listen(FBUtilities.getLocalAddressAndPort(), serverEncryptionOptions);
         if (shouldListenOnBroadcastAddress())
-            listen(FBUtilities.getBroadcastAddress(), serverEncryptionOptions);
+            listen(FBUtilities.getBroadcastAddressAndPort(), serverEncryptionOptions);
         listenGate.signalAll();
     }
 
     public static boolean shouldListenOnBroadcastAddress()
     {
         return DatabaseDescriptor.shouldListenOnBroadcastAddress()
-               && !FBUtilities.getLocalAddress().equals(FBUtilities.getBroadcastAddress());
+               && !FBUtilities.getLocalAddressAndPort().equals(FBUtilities.getBroadcastAddressAndPort());
     }
 
     /**
      * Listen on the specified port.
      *
-     * @param localEp InetAddress whose port to listen on.
+     * @param localEp InetAddressAndPort whose port to listen on.
      */
-    private void listen(InetAddress localEp, ServerEncryptionOptions serverEncryptionOptions) throws ConfigurationException
+    private void listen(InetAddressAndPort localEp, ServerEncryptionOptions serverEncryptionOptions) throws ConfigurationException
     {
         IInternodeAuthenticator authenticator = DatabaseDescriptor.getInternodeAuthenticator();
         int receiveBufferSize = DatabaseDescriptor.getInternodeRecvBufferSize();
@@ -766,7 +763,7 @@ public final class MessagingService implements MessagingServiceMBean
             ServerEncryptionOptions legacyEncOptions = new ServerEncryptionOptions(serverEncryptionOptions);
             legacyEncOptions.optional = false;
 
-            InetSocketAddress localAddr = new InetSocketAddress(localEp, DatabaseDescriptor.getSSLStoragePort());
+            InetAddressAndPort localAddr = InetAddressAndPort.getByAddressOverrideDefaults(localEp.address, DatabaseDescriptor.getSSLStoragePort());
             ChannelGroup channelGroup = new DefaultChannelGroup("LegacyEncryptedInternodeMessagingGroup", NettyFactory.executorForChannelGroups());
             InboundInitializer initializer = new InboundInitializer(authenticator, legacyEncOptions, channelGroup);
             Channel encryptedChannel = NettyFactory.instance.createInboundChannel(localAddr, initializer, receiveBufferSize);
@@ -774,7 +771,8 @@ public final class MessagingService implements MessagingServiceMBean
         }
 
         // this is for the socket that can be plain, only ssl, or optional plain/ssl
-        InetSocketAddress localAddr = new InetSocketAddress(localEp, DatabaseDescriptor.getStoragePort());
+        assert localEp.port == DatabaseDescriptor.getStoragePort() : String.format("Local endpoint port %d doesn't match YAML configured port %d%n", localEp.port, DatabaseDescriptor.getStoragePort());
+        InetAddressAndPort localAddr = InetAddressAndPort.getByAddressOverrideDefaults(localEp.address, DatabaseDescriptor.getStoragePort());
         ChannelGroup channelGroup = new DefaultChannelGroup("InternodeMessagingGroup", NettyFactory.executorForChannelGroups());
         InboundInitializer initializer = new InboundInitializer(authenticator, serverEncryptionOptions, channelGroup);
         Channel channel = NettyFactory.instance.createInboundChannel(localAddr, initializer, receiveBufferSize);
@@ -809,10 +807,10 @@ public final class MessagingService implements MessagingServiceMBean
          * the inbound connections/channels can be closed when the listening socket itself is being closed.
          */
         private final ChannelGroup connectedChannels;
-        private final InetSocketAddress address;
+        private final InetAddressAndPort address;
         private final SecurityLevel securityLevel;
 
-        private ServerChannel(Channel channel, ChannelGroup channelGroup, InetSocketAddress address, SecurityLevel securityLevel)
+        private ServerChannel(Channel channel, ChannelGroup channelGroup, InetAddressAndPort address, SecurityLevel securityLevel)
         {
             this.channel = channel;
             this.connectedChannels = channelGroup;
@@ -840,7 +838,7 @@ public final class MessagingService implements MessagingServiceMBean
             return channel;
         }
 
-        InetSocketAddress getAddress()
+        InetAddressAndPort getAddress()
         {
             return address;
         }
@@ -869,7 +867,7 @@ public final class MessagingService implements MessagingServiceMBean
     }
 
 
-    public void destroyConnectionPool(InetAddress to)
+    public void destroyConnectionPool(InetAddressAndPort to)
     {
         OutboundMessagingPool pool = channelManagers.remove(to);
         if (pool != null)
@@ -884,26 +882,26 @@ public final class MessagingService implements MessagingServiceMBean
      * @param address IP Address to identify the peer
      * @param preferredAddress IP Address to use (and prefer) going forward for connecting to the peer
      */
-    public void reconnectWithNewIp(InetAddress address, InetAddress preferredAddress)
+    public void reconnectWithNewIp(InetAddressAndPort address, InetAddressAndPort preferredAddress)
     {
         SystemKeyspace.updatePreferredIP(address, preferredAddress);
 
         OutboundMessagingPool messagingPool = channelManagers.get(address);
         if (messagingPool != null)
-            messagingPool.reconnectWithNewIp(new InetSocketAddress(preferredAddress, portFor(address)));
+            messagingPool.reconnectWithNewIp(InetAddressAndPort.getByAddressOverrideDefaults(preferredAddress.address, portFor(address)));
     }
 
-    private void reset(InetAddress address)
+    private void reset(InetAddressAndPort address)
     {
         OutboundMessagingPool messagingPool = channelManagers.remove(address);
         if (messagingPool != null)
             messagingPool.close(false);
     }
 
-    public InetAddress getCurrentEndpoint(InetAddress publicAddress)
+    public InetAddressAndPort getCurrentEndpoint(InetAddressAndPort publicAddress)
     {
         OutboundMessagingPool messagingPool = getMessagingConnection(publicAddress);
-        return messagingPool != null ? messagingPool.getPreferredRemoteAddr().getAddress() : null;
+        return messagingPool != null ? messagingPool.getPreferredRemoteAddr() : null;
     }
 
     /**
@@ -931,7 +929,7 @@ public final class MessagingService implements MessagingServiceMBean
         return verbHandlers.get(type);
     }
 
-    public int addCallback(IAsyncCallback cb, MessageOut message, InetAddress to, long timeout, boolean failureCallback)
+    public int addCallback(IAsyncCallback cb, MessageOut message, InetAddressAndPort to, long timeout, boolean failureCallback)
     {
         assert message.verb != Verb.MUTATION; // mutations need to call the overload with a ConsistencyLevel
         int messageId = nextId();
@@ -942,7 +940,7 @@ public final class MessagingService implements MessagingServiceMBean
 
     public int addCallback(IAsyncCallback cb,
                            MessageOut<?> message,
-                           InetAddress to,
+                           InetAddressAndPort to,
                            long timeout,
                            ConsistencyLevel consistencyLevel,
                            boolean allowHints)
@@ -971,12 +969,12 @@ public final class MessagingService implements MessagingServiceMBean
         return idGen.incrementAndGet();
     }
 
-    public int sendRR(MessageOut message, InetAddress to, IAsyncCallback cb)
+    public int sendRR(MessageOut message, InetAddressAndPort to, IAsyncCallback cb)
     {
         return sendRR(message, to, cb, message.getTimeout(), false);
     }
 
-    public int sendRRWithFailure(MessageOut message, InetAddress to, IAsyncCallbackWithFailure cb)
+    public int sendRRWithFailure(MessageOut message, InetAddressAndPort to, IAsyncCallbackWithFailure cb)
     {
         return sendRR(message, to, cb, message.getTimeout(), true);
     }
@@ -992,11 +990,11 @@ public final class MessagingService implements MessagingServiceMBean
      * @param timeout the timeout used for expiration
      * @return an reference to message id used to match with the result
      */
-    public int sendRR(MessageOut message, InetAddress to, IAsyncCallback cb, long timeout, boolean failureCallback)
+    public int sendRR(MessageOut message, InetAddressAndPort to, IAsyncCallback cb, long timeout, boolean failureCallback)
     {
         int id = addCallback(cb, message, to, timeout, failureCallback);
         updateBackPressureOnSend(to, cb, message);
-        sendOneWay(failureCallback ? message.withParameter(FAILURE_CALLBACK_PARAM, ONE_BYTE) : message, id, to);
+        sendOneWay(failureCallback ? message.withParameter(ParameterType.FAILURE_CALLBACK, ONE_BYTE) : message, id, to);
         return id;
     }
 
@@ -1013,22 +1011,22 @@ public final class MessagingService implements MessagingServiceMBean
      * @return an reference to message id used to match with the result
      */
     public int sendRR(MessageOut<?> message,
-                      InetAddress to,
+                      InetAddressAndPort to,
                       AbstractWriteResponseHandler<?> handler,
                       boolean allowHints)
     {
         int id = addCallback(handler, message, to, message.getTimeout(), handler.consistencyLevel, allowHints);
         updateBackPressureOnSend(to, handler, message);
-        sendOneWay(message.withParameter(FAILURE_CALLBACK_PARAM, ONE_BYTE), id, to);
+        sendOneWay(message.withParameter(ParameterType.FAILURE_CALLBACK, ONE_BYTE), id, to);
         return id;
     }
 
-    public void sendOneWay(MessageOut message, InetAddress to)
+    public void sendOneWay(MessageOut message, InetAddressAndPort to)
     {
         sendOneWay(message, nextId(), to);
     }
 
-    public void sendReply(MessageOut message, int id, InetAddress to)
+    public void sendReply(MessageOut message, int id, InetAddressAndPort to)
     {
         sendOneWay(message, id, to);
     }
@@ -1040,12 +1038,12 @@ public final class MessagingService implements MessagingServiceMBean
      * @param message messages to be sent.
      * @param to      endpoint to which the message needs to be sent
      */
-    public void sendOneWay(MessageOut message, int id, InetAddress to)
+    public void sendOneWay(MessageOut message, int id, InetAddressAndPort to)
     {
         if (logger.isTraceEnabled())
-            logger.trace("{} sending {} to {}@{}", FBUtilities.getBroadcastAddress(), message.verb, id, to);
+            logger.trace("{} sending {} to {}@{}", FBUtilities.getBroadcastAddressAndPort(), message.verb, id, to);
 
-        if (to.equals(FBUtilities.getBroadcastAddress()))
+        if (to.equals(FBUtilities.getBroadcastAddressAndPort()))
             logger.trace("Message-to-self {} going over MessagingService", message);
 
         // message sinks are a testing hook
@@ -1058,7 +1056,7 @@ public final class MessagingService implements MessagingServiceMBean
             outboundMessagingPool.sendMessage(message, id);
     }
 
-    public <T> AsyncOneResponse<T> sendRR(MessageOut message, InetAddress to)
+    public <T> AsyncOneResponse<T> sendRR(MessageOut message, InetAddressAndPort to)
     {
         AsyncOneResponse<T> iar = new AsyncOneResponse<T>();
         sendRR(message, to, iar);
@@ -1176,7 +1174,7 @@ public final class MessagingService implements MessagingServiceMBean
     /**
      * @return the last version associated with address, or @param version if this is the first such version
      */
-    public int setVersion(InetAddress endpoint, int version)
+    public int setVersion(InetAddressAndPort endpoint, int version)
     {
         logger.trace("Setting version {} for {}", version, endpoint);
 
@@ -1184,7 +1182,7 @@ public final class MessagingService implements MessagingServiceMBean
         return v == null ? version : v;
     }
 
-    public void resetVersion(InetAddress endpoint)
+    public void resetVersion(InetAddressAndPort endpoint)
     {
         logger.trace("Resetting version for {}", endpoint);
         versions.remove(endpoint);
@@ -1194,7 +1192,7 @@ public final class MessagingService implements MessagingServiceMBean
      * Returns the messaging-version as announced by the given node but capped
      * to the min of the version as announced by the node and {@link #current_version}.
      */
-    public int getVersion(InetAddress endpoint)
+    public int getVersion(InetAddressAndPort endpoint)
     {
         Integer v = versions.get(endpoint);
         if (v == null)
@@ -1209,13 +1207,13 @@ public final class MessagingService implements MessagingServiceMBean
 
     public int getVersion(String endpoint) throws UnknownHostException
     {
-        return getVersion(InetAddress.getByName(endpoint));
+        return getVersion(InetAddressAndPort.getByName(endpoint));
     }
 
     /**
      * Returns the messaging-version exactly as announced by the given endpoint.
      */
-    public int getRawVersion(InetAddress endpoint)
+    public int getRawVersion(InetAddressAndPort endpoint)
     {
         Integer v = versions.get(endpoint);
         if (v == null)
@@ -1223,7 +1221,7 @@ public final class MessagingService implements MessagingServiceMBean
         return v;
     }
 
-    public boolean knowsVersion(InetAddress endpoint)
+    public boolean knowsVersion(InetAddressAndPort endpoint)
     {
         return versions.containsKey(endpoint);
     }
@@ -1358,72 +1356,144 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Integer> getLargeMessagePendingTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().toString(false), entry.getValue().largeMessageChannel.getPendingMessages());
         return pendingTasks;
     }
 
     public Map<String, Long> getLargeMessageCompletedTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().toString(false), entry.getValue().largeMessageChannel.getCompletedMessages());
         return completedTasks;
     }
 
     public Map<String, Long> getLargeMessageDroppedTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().toString(false), entry.getValue().largeMessageChannel.getDroppedMessages());
         return droppedTasks;
     }
 
     public Map<String, Integer> getSmallMessagePendingTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().toString(false), entry.getValue().smallMessageChannel.getPendingMessages());
         return pendingTasks;
     }
 
     public Map<String, Long> getSmallMessageCompletedTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().toString(false), entry.getValue().smallMessageChannel.getCompletedMessages());
         return completedTasks;
     }
 
     public Map<String, Long> getSmallMessageDroppedTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().toString(false), entry.getValue().smallMessageChannel.getDroppedMessages());
         return droppedTasks;
     }
 
     public Map<String, Integer> getGossipMessagePendingTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().toString(false), entry.getValue().gossipChannel.getPendingMessages());
         return pendingTasks;
     }
 
     public Map<String, Long> getGossipMessageCompletedTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().toString(false), entry.getValue().gossipChannel.getCompletedMessages());
         return completedTasks;
     }
 
     public Map<String, Long> getGossipMessageDroppedTasks()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().toString(false), entry.getValue().gossipChannel.getDroppedMessages());
+        return droppedTasks;
+    }
+
+    public Map<String, Integer> getLargeMessagePendingTasksWithPort()
+    {
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().toString(), entry.getValue().largeMessageChannel.getPendingMessages());
+        return pendingTasks;
+    }
+
+    public Map<String, Long> getLargeMessageCompletedTasksWithPort()
+    {
+        Map<String, Long> completedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().toString(), entry.getValue().largeMessageChannel.getCompletedMessages());
+        return completedTasks;
+    }
+
+    public Map<String, Long> getLargeMessageDroppedTasksWithPort()
+    {
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().toString(), entry.getValue().largeMessageChannel.getDroppedMessages());
+        return droppedTasks;
+    }
+
+    public Map<String, Integer> getSmallMessagePendingTasksWithPort()
+    {
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().toString(), entry.getValue().smallMessageChannel.getPendingMessages());
+        return pendingTasks;
+    }
+
+    public Map<String, Long> getSmallMessageCompletedTasksWithPort()
+    {
+        Map<String, Long> completedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().toString(), entry.getValue().smallMessageChannel.getCompletedMessages());
+        return completedTasks;
+    }
+
+    public Map<String, Long> getSmallMessageDroppedTasksWithPort()
+    {
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().toString(), entry.getValue().smallMessageChannel.getDroppedMessages());
+        return droppedTasks;
+    }
+
+    public Map<String, Integer> getGossipMessagePendingTasksWithPort()
+    {
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            pendingTasks.put(entry.getKey().toString(), entry.getValue().gossipChannel.getPendingMessages());
+        return pendingTasks;
+    }
+
+    public Map<String, Long> getGossipMessageCompletedTasksWithPort()
+    {
+        Map<String, Long> completedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            completedTasks.put(entry.getKey().toString(), entry.getValue().gossipChannel.getCompletedMessages());
+        return completedTasks;
+    }
+
+    public Map<String, Long> getGossipMessageDroppedTasksWithPort()
+    {
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            droppedTasks.put(entry.getKey().toString(), entry.getValue().gossipChannel.getDroppedMessages());
         return droppedTasks;
     }
 
@@ -1435,7 +1505,6 @@ public final class MessagingService implements MessagingServiceMBean
         return map;
     }
 
-
     public long getTotalTimeouts()
     {
         return ConnectionMetrics.totalTimeouts.getCount();
@@ -1444,9 +1513,21 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Long> getTimeoutsPerHost()
     {
         Map<String, Long> result = new HashMap<String, Long>(channelManagers.size());
-        for (Map.Entry<InetAddress, OutboundMessagingPool> entry : channelManagers.entrySet())
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
         {
-            String ip = entry.getKey().getHostAddress();
+            String ip = entry.getKey().toString(false);
+            long recent = entry.getValue().getTimeouts();
+            result.put(ip, recent);
+        }
+        return result;
+    }
+
+    public Map<String, Long> getTimeoutsPerHostWithPort()
+    {
+        Map<String, Long> result = new HashMap<String, Long>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+        {
+            String ip = entry.getKey().toString();
             long recent = entry.getValue().getTimeouts();
             result.put(ip, recent);
         }
@@ -1456,8 +1537,17 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Double> getBackPressurePerHost()
     {
         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());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            map.put(entry.getKey().toString(false), entry.getValue().getBackPressureState().getBackPressureRateLimit());
+
+        return map;
+    }
+
+    public Map<String, Double> getBackPressurePerHostWithPort()
+    {
+        Map<String, Double> map = new HashMap<>(channelManagers.size());
+        for (Map.Entry<InetAddressAndPort, OutboundMessagingPool> entry : channelManagers.entrySet())
+            map.put(entry.getKey().toString(false), entry.getValue().getBackPressureState().getBackPressureRateLimit());
 
         return map;
     }
@@ -1493,18 +1583,18 @@ public final class MessagingService implements MessagingServiceMBean
                                                    bounds.left.getPartitioner().getClass().getName()));
     }
 
-    private OutboundMessagingPool getMessagingConnection(InetAddress to)
+    private OutboundMessagingPool getMessagingConnection(InetAddressAndPort to)
     {
         OutboundMessagingPool pool = channelManagers.get(to);
         if (pool == null)
         {
             final boolean secure = isEncryptedConnection(to);
             final int port = portFor(to, secure);
-            if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(to, port))
+            if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(to.address, port))
                 return null;
 
-            InetSocketAddress preferredRemote = new InetSocketAddress(SystemKeyspace.getPreferredIP(to), port);
-            InetSocketAddress local = new InetSocketAddress(FBUtilities.getLocalAddress(), 0);
+            InetAddressAndPort preferredRemote = SystemKeyspace.getPreferredIP(to);
+            InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
             ServerEncryptionOptions encryptionOptions = secure ? DatabaseDescriptor.getServerEncryptionOptions() : null;
             IInternodeAuthenticator authenticator = DatabaseDescriptor.getInternodeAuthenticator();
 
@@ -1519,16 +1609,16 @@ public final class MessagingService implements MessagingServiceMBean
         return pool;
     }
 
-    public int portFor(InetAddress addr)
+    public int portFor(InetAddressAndPort addr)
     {
         final boolean secure = isEncryptedConnection(addr);
         return portFor(addr, secure);
     }
 
-    private int portFor(InetAddress address, boolean secure)
+    private int portFor(InetAddressAndPort address, boolean secure)
     {
         if (!secure)
-            return DatabaseDescriptor.getStoragePort();
+            return address.port;
 
         Integer v = versions.get(address);
         // if we don't know the version of the peer, assume it is 4.0 (or higher) as the only time is would be lower
@@ -1536,12 +1626,15 @@ public final class MessagingService implements MessagingServiceMBean
         // unfortunately fail - however the peer should connect to this node (at some point), and once we learn it's version, it'll be
         // in versions map. thus, when we attempt to reconnect to that node, we'll have the version and we can get the correct port.
         // we will be able to remove this logic at 5.0.
+        // Also as of 4.0 we will propagate the "regular" port (which will support both SSL and non-SSL) via gossip so
+        // for SSL and version 4.0 always connect to the gossiped port because if SSL is enabled it should ALWAYS
+        // listen for SSL on the "regular" port.
         int version = v != null ? v.intValue() : VERSION_40;
-        return version < VERSION_40 ? DatabaseDescriptor.getSSLStoragePort() : DatabaseDescriptor.getStoragePort();
+        return version < VERSION_40 ? DatabaseDescriptor.getSSLStoragePort() : address.port;
     }
 
     @VisibleForTesting
-    boolean isConnected(InetAddress address, MessageOut messageOut)
+    boolean isConnected(InetAddressAndPort address, MessageOut messageOut)
     {
         OutboundMessagingPool pool = channelManagers.get(address);
         if (pool == null)
@@ -1549,7 +1642,7 @@ public final class MessagingService implements MessagingServiceMBean
         return pool.getConnection(messageOut).isConnected();
     }
 
-    public static boolean isEncryptedConnection(InetAddress address)
+    public static boolean isEncryptedConnection(InetAddressAndPort address)
     {
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
         switch (DatabaseDescriptor.getServerEncryptionOptions().internode_encryption)
@@ -1559,13 +1652,13 @@ public final class MessagingService implements MessagingServiceMBean
             case all:
                 break;
             case dc:
-                if (snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddress())))
+                if (snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddressAndPort())))
                     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())))
+                if (snitch.getRack(address).equals(snitch.getRack(FBUtilities.getBroadcastAddressAndPort()))
+                    && snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddressAndPort())))
                     return false;
                 break;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingServiceMBean.java b/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
index b2e79e0..f4a0c43 100644
--- a/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
+++ b/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
@@ -30,47 +30,69 @@ public interface MessagingServiceMBean
     /**
      * Pending tasks for large message TCP Connections
      */
+    @Deprecated
     public Map<String, Integer> getLargeMessagePendingTasks();
+    public Map<String, Integer> getLargeMessagePendingTasksWithPort();
 
     /**
      * Completed tasks for large message) TCP Connections
      */
+    @Deprecated
     public Map<String, Long> getLargeMessageCompletedTasks();
+    public Map<String, Long> getLargeMessageCompletedTasksWithPort();
 
     /**
      * Dropped tasks for large message TCP Connections
      */
+    @Deprecated
     public Map<String, Long> getLargeMessageDroppedTasks();
+    public Map<String, Long> getLargeMessageDroppedTasksWithPort();
+
 
     /**
      * Pending tasks for small message TCP Connections
      */
+    @Deprecated
     public Map<String, Integer> getSmallMessagePendingTasks();
+    public Map<String, Integer> getSmallMessagePendingTasksWithPort();
+
 
     /**
      * Completed tasks for small message TCP Connections
      */
+    @Deprecated
     public Map<String, Long> getSmallMessageCompletedTasks();
+    public Map<String, Long> getSmallMessageCompletedTasksWithPort();
+
 
     /**
      * Dropped tasks for small message TCP Connections
      */
+    @Deprecated
     public Map<String, Long> getSmallMessageDroppedTasks();
+    public Map<String, Long> getSmallMessageDroppedTasksWithPort();
+
 
     /**
      * Pending tasks for gossip message TCP Connections
      */
+    @Deprecated
     public Map<String, Integer> getGossipMessagePendingTasks();
+    public Map<String, Integer> getGossipMessagePendingTasksWithPort();
 
     /**
      * Completed tasks for gossip message TCP Connections
      */
+    @Deprecated
     public Map<String, Long> getGossipMessageCompletedTasks();
+    public Map<String, Long> getGossipMessageCompletedTasksWithPort();
 
     /**
      * Dropped tasks for gossip message TCP Connections
      */
+    @Deprecated
     public Map<String, Long> getGossipMessageDroppedTasks();
+    public Map<String, Long> getGossipMessageDroppedTasksWithPort();
 
     /**
      * dropped message counts for server lifetime
@@ -85,12 +107,16 @@ public interface MessagingServiceMBean
     /**
      * Number of timeouts per host
      */
+    @Deprecated
     public Map<String, Long> getTimeoutsPerHost();
+    public Map<String, Long> getTimeoutsPerHostWithPort();
 
     /**
      * Back-pressure rate limiting per host
      */
+    @Deprecated
     public Map<String, Double> getBackPressurePerHost();
+    public Map<String, Double> getBackPressurePerHostWithPort();
 
     /**
      * Enable/Disable back-pressure

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/ParameterType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/ParameterType.java b/src/java/org/apache/cassandra/net/ParameterType.java
new file mode 100644
index 0000000..0a1f73f
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/ParameterType.java
@@ -0,0 +1,69 @@
+/*
+ * 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.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.io.DummyByteVersionedSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.ShortVersionedSerializer;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+/**
+ * Type names and serializers for various parameters that
+ */
+public enum ParameterType
+{
+    FORWARD_TO("FORWARD_TO", ForwardToSerializer.instance),
+    FORWARD_FROM("FORWARD_FROM", CompactEndpointSerializationHelper.instance),
+    FAILURE_RESPONSE("FAIL", DummyByteVersionedSerializer.instance),
+    FAILURE_REASON("FAIL_REASON", ShortVersionedSerializer.instance),
+    FAILURE_CALLBACK("CAL_BAC", DummyByteVersionedSerializer.instance),
+    TRACE_SESSION("TraceSession", UUIDSerializer.serializer),
+    TRACE_TYPE("TraceType", Tracing.traceTypeSerializer);
+
+    public static final Map<String, ParameterType> byName;
+    public final String key;
+    public final IVersionedSerializer serializer;
+
+    static
+    {
+        ImmutableMap.Builder<String, ParameterType> builder = ImmutableMap.builder();
+        for (ParameterType type : values())
+        {
+            builder.put(type.key, type);
+        }
+        byName = builder.build();
+    }
+
+    ParameterType(String key, IVersionedSerializer serializer)
+    {
+        this.key = key;
+        this.serializer = serializer;
+    }
+
+    public String key()
+    {
+        return key;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/RateBasedBackPressure.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/RateBasedBackPressure.java b/src/java/org/apache/cassandra/net/RateBasedBackPressure.java
index 64685b0..b951bc0 100644
--- a/src/java/org/apache/cassandra/net/RateBasedBackPressure.java
+++ b/src/java/org/apache/cassandra/net/RateBasedBackPressure.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -34,6 +33,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.SystemTimeSource;
 import org.apache.cassandra.utils.TimeSource;
@@ -253,7 +253,7 @@ public class RateBasedBackPressure implements BackPressureStrategy<RateBasedBack
     }
 
     @Override
-    public RateBasedBackPressureState newState(InetAddress host)
+    public RateBasedBackPressureState newState(InetAddressAndPort host)
     {
         return new RateBasedBackPressureState(host, timeSource, windowSize);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java b/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java
index 541d7a6..9df056e 100644
--- a/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java
+++ b/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java
@@ -17,11 +17,11 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.util.concurrent.RateLimiter;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.SlidingTimeRate;
 import org.apache.cassandra.utils.TimeSource;
 import org.apache.cassandra.utils.concurrent.IntervalLock;
@@ -46,12 +46,12 @@ import org.apache.cassandra.utils.concurrent.IntervalLock;
  */
 class RateBasedBackPressureState extends IntervalLock implements BackPressureState
 {
-    private final InetAddress host;
+    private final InetAddressAndPort host;
     final SlidingTimeRate incomingRate;
     final SlidingTimeRate outgoingRate;
     final RateLimiter rateLimiter;
 
-    RateBasedBackPressureState(InetAddress host, TimeSource timeSource, long windowSize)
+    RateBasedBackPressureState(InetAddressAndPort host, TimeSource timeSource, long windowSize)
     {
         super(timeSource);
         this.host = host;
@@ -99,7 +99,7 @@ class RateBasedBackPressureState extends IntervalLock implements BackPressureSta
     }
 
     @Override
-    public InetAddress getHost()
+    public InetAddressAndPort getHost()
     {
         return host;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/WriteCallbackInfo.java b/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
index 9ecc385..41ac31b 100644
--- a/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
+++ b/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
@@ -18,11 +18,10 @@
  */
 package org.apache.cassandra.net;
 
-import java.net.InetAddress;
-
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.utils.FBUtilities;
@@ -32,7 +31,7 @@ public class WriteCallbackInfo extends CallbackInfo
     // either a Mutation, or a Paxos Commit (MessageOut)
     private final Object mutation;
 
-    public WriteCallbackInfo(InetAddress target,
+    public WriteCallbackInfo(InetAddressAndPort target,
                              IAsyncCallback callback,
                              MessageOut message,
                              IVersionedSerializer<?> serializer,
@@ -43,7 +42,7 @@ public class WriteCallbackInfo extends CallbackInfo
         assert message != null;
         this.mutation = shouldHint(allowHints, message, consistencyLevel);
         //Local writes shouldn't go through messaging service (https://issues.apache.org/jira/browse/CASSANDRA-10477)
-        assert (!target.equals(FBUtilities.getBroadcastAddress()));
+        assert (!target.equals(FBUtilities.getBroadcastAddressAndPort()));
     }
 
     public boolean shouldHint()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index 9b8df80..327b20e 100644
--- a/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
+++ b/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
@@ -24,11 +24,15 @@ import java.net.InetAddress;
 import java.util.Objects;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.primitives.Ints;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.ByteBufInputStream;
 import io.netty.buffer.ByteBufOutputStream;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
 
@@ -227,9 +231,9 @@ public class HandshakeProtocol
         private static final int MIN_LENGTH = 9;
 
         final int messagingVersion;
-        final InetAddress address;
+        final InetAddressAndPort address;
 
-        ThirdHandshakeMessage(int messagingVersion, InetAddress address)
+        ThirdHandshakeMessage(int messagingVersion, InetAddressAndPort address)
         {
             this.messagingVersion = messagingVersion;
             this.address = address;
@@ -238,14 +242,14 @@ public class HandshakeProtocol
         @SuppressWarnings("resource")
         public ByteBuf encode(ByteBufAllocator allocator)
         {
-            int bufLength = Integer.BYTES + CompactEndpointSerializationHelper.serializedSize(address);
+            int bufLength = Ints.checkedCast(Integer.BYTES + CompactEndpointSerializationHelper.instance.serializedSize(address, messagingVersion));
             ByteBuf buffer = allocator.directBuffer(bufLength, bufLength);
             buffer.writerIndex(0);
             buffer.writeInt(messagingVersion);
             try
             {
-                DataOutput bbos = new ByteBufOutputStream(buffer);
-                CompactEndpointSerializationHelper.serialize(address, bbos);
+                DataOutputPlus dop = new ByteBufDataOutputPlus(buffer);
+                CompactEndpointSerializationHelper.instance.serialize(address, dop, messagingVersion);
                 return buffer;
             }
             catch (IOException e)
@@ -263,10 +267,10 @@ public class HandshakeProtocol
 
             in.markReaderIndex();
             int version = in.readInt();
-            DataInput inputStream = new ByteBufInputStream(in);
+            DataInputPlus input = new ByteBufDataInputPlus(in);
             try
             {
-                InetAddress address = CompactEndpointSerializationHelper.deserialize(inputStream);
+                InetAddressAndPort address = CompactEndpointSerializationHelper.instance.deserialize(input, version);
                 return new ThirdHandshakeMessage(version, address);
             }
             catch (IOException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 625f03d..a84112e 100644
--- a/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
+++ b/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java
@@ -22,6 +22,7 @@ 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.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage;
 import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage;
@@ -209,7 +210,7 @@ class InboundHandshakeHandler extends ByteToMessageDecoder
     {
         ChannelPipeline pipeline = ctx.pipeline();
         InetSocketAddress address = (InetSocketAddress) ctx.channel().remoteAddress();
-        pipeline.addLast(NettyFactory.instance.streamingGroup, "streamInbound", new StreamingInboundHandler(address, protocolVersion, null));
+        pipeline.addLast(NettyFactory.instance.streamingGroup, "streamInbound", new StreamingInboundHandler(InetAddressAndPort.getByAddressOverrideDefaults(address.getAddress(), address.getPort()), 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
@@ -244,7 +245,7 @@ class InboundHandshakeHandler extends ByteToMessageDecoder
         }
 
         // record the (true) version of the endpoint
-        InetAddress from = msg.address;
+        InetAddressAndPort from = msg.address;
         MessagingService.instance().setVersion(from, maxVersion);
         logger.trace("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from));
 
@@ -253,7 +254,7 @@ class InboundHandshakeHandler extends ByteToMessageDecoder
     }
 
     @VisibleForTesting
-    void setupMessagingPipeline(ChannelPipeline pipeline, InetAddress peer, boolean compressed, int messagingVersion)
+    void setupMessagingPipeline(ChannelPipeline pipeline, InetAddressAndPort peer, boolean compressed, int messagingVersion)
     {
         if (compressed)
             pipeline.addLast(NettyFactory.INBOUND_COMPRESSOR_HANDLER_NAME, NettyFactory.createLz4Decoder(messagingVersion));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index b400512..0423b80 100644
--- a/src/java/org/apache/cassandra/net/async/MessageInHandler.java
+++ b/src/java/org/apache/cassandra/net/async/MessageInHandler.java
@@ -21,7 +21,6 @@ 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;
@@ -37,9 +36,12 @@ 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.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.ParameterType;
 
 /**
  * Parses out individual messages from the incoming buffers. Each message, both header and payload, is incrementally built up
@@ -79,7 +81,7 @@ class MessageInHandler extends ByteToMessageDecoder
      */
     private static final int SECOND_SECTION_BYTE_COUNT = 8;
 
-    private final InetAddress peer;
+    private final InetAddressAndPort peer;
     private final int messagingVersion;
 
     /**
@@ -91,12 +93,12 @@ class MessageInHandler extends ByteToMessageDecoder
     private State state;
     private MessageHeader messageHeader;
 
-    MessageInHandler(InetAddress peer, int messagingVersion)
+    MessageInHandler(InetAddressAndPort peer, int messagingVersion)
     {
         this (peer, messagingVersion, MESSAGING_SERVICE_CONSUMER);
     }
 
-    MessageInHandler(InetAddress peer, int messagingVersion, BiConsumer<MessageIn, Integer> messageConsumer)
+    MessageInHandler(InetAddressAndPort peer, int messagingVersion, BiConsumer<MessageIn, Integer> messageConsumer)
     {
         this.peer = peer;
         this.messagingVersion = messagingVersion;
@@ -140,7 +142,7 @@ class MessageInHandler extends ByteToMessageDecoder
                         int serializedAddrSize;
                         if (readableBytes < 1 || readableBytes < (serializedAddrSize = in.getByte(in.readerIndex()) + 1))
                             return;
-                        messageHeader.from = CompactEndpointSerializationHelper.deserialize(inputPlus);
+                        messageHeader.from = CompactEndpointSerializationHelper.instance.deserialize(inputPlus, messagingVersion);
                         state = State.READ_SECOND_CHUNK;
                         readableBytes -= serializedAddrSize;
                         // fall-through
@@ -199,7 +201,7 @@ class MessageInHandler extends ByteToMessageDecoder
     /**
      * @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
+    private boolean readParameters(ByteBuf in, ByteBufDataInputPlus inputPlus, int parameterCount, Map<ParameterType, Object> parameters) throws IOException
     {
         // makes the assumption that map.size() is a constant time function (HashMap.size() is)
         while (parameters.size() < parameterCount)
@@ -208,9 +210,10 @@ class MessageInHandler extends ByteToMessageDecoder
                 return false;
 
             String key = DataInputStream.readUTF(inputPlus);
+            ParameterType parameterType = ParameterType.byName.get(key);
             byte[] value = new byte[in.readInt()];
             in.readBytes(value);
-            parameters.put(key, value);
+            parameters.put(parameterType, parameterType.serializer.deserialize(new DataInputBuffer(value), messagingVersion));
         }
 
         return true;
@@ -300,11 +303,11 @@ class MessageInHandler extends ByteToMessageDecoder
     {
         int messageId;
         long constructionTime;
-        InetAddress from;
+        InetAddressAndPort from;
         MessagingService.Verb verb;
         int payloadSize;
 
-        Map<String, byte[]> parameters = Collections.emptyMap();
+        Map<ParameterType, Object> parameters = Collections.emptyMap();
 
         /**
          * Total number of incoming parameters.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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
index e88b56a..f1647ab 100644
--- a/src/java/org/apache/cassandra/net/async/MessageOutHandler.java
+++ b/src/java/org/apache/cassandra/net/async/MessageOutHandler.java
@@ -40,6 +40,7 @@ import io.netty.handler.timeout.IdleStateEvent;
 
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.ParameterType;
 import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis;
@@ -196,10 +197,9 @@ class MessageOutHandler extends ChannelDuplexHandler
     {
         try
         {
-            byte[] sessionBytes = msg.message.parameters.get(Tracing.TRACE_HEADER);
-            if (sessionBytes != null)
+            UUID sessionId =  (UUID)msg.message.getParameter(ParameterType.TRACE_SESSION);
+            if (sessionId != 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(),
@@ -207,9 +207,9 @@ class MessageOutHandler extends ChannelDuplexHandler
                 // 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());
+                    Tracing.TraceType traceType = (Tracing.TraceType)msg.message.getParameter(ParameterType.TRACE_TYPE);
+                    traceType = traceType == null ? Tracing.TraceType.QUERY : traceType;
+                    Tracing.instance.trace(ByteBuffer.wrap(UUIDGen.decompose(sessionId)), message, traceType.getTTL());
                 }
                 else
                 {


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


[06/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
index 84a4c32..52fc6ac 100644
--- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
+++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
@@ -19,10 +19,14 @@ package org.apache.cassandra.utils;
 
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
 import java.util.*;
+import java.util.stream.Collectors;
 
 import com.datastax.driver.core.*;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder;
@@ -34,33 +38,46 @@ import org.apache.cassandra.dht.Token.TokenFactory;
 import org.apache.cassandra.io.sstable.SSTableLoader;
 import org.apache.cassandra.schema.TableMetadata;
 
+import org.apache.cassandra.schema.CQLTypeParser;
+import org.apache.cassandra.schema.SchemaKeyspace;
+import org.apache.cassandra.schema.Types;
+
 public class NativeSSTableLoaderClient extends SSTableLoader.Client
 {
     protected final Map<String, TableMetadataRef> tables;
-    private final Collection<InetAddress> hosts;
+    private final Collection<InetSocketAddress> hosts;
     private final int port;
+    private final int storagePort;
     private final AuthProvider authProvider;
     private final SSLOptions sslOptions;
+    private final boolean allowServerPortDiscovery;
 
 
-    public NativeSSTableLoaderClient(Collection<InetAddress> hosts, int port, String username, String password, SSLOptions sslOptions)
+    public NativeSSTableLoaderClient(Collection<InetSocketAddress> hosts, int nativePort, int storagePort, String username, String password, SSLOptions sslOptions, boolean allowServerPortDiscovery)
     {
-        this(hosts, port, new PlainTextAuthProvider(username, password), sslOptions);
+        this(hosts, nativePort, storagePort, new PlainTextAuthProvider(username, password), sslOptions, allowServerPortDiscovery);
     }
 
-    public NativeSSTableLoaderClient(Collection<InetAddress> hosts, int port, AuthProvider authProvider, SSLOptions sslOptions)
+    public NativeSSTableLoaderClient(Collection<InetSocketAddress> hosts, int nativePort, int storagePort, AuthProvider authProvider, SSLOptions sslOptions, boolean allowServerPortDiscovery)
     {
         super();
         this.tables = new HashMap<>();
         this.hosts = hosts;
-        this.port = port;
+        this.port = nativePort;
         this.authProvider = authProvider;
         this.sslOptions = sslOptions;
+        this.allowServerPortDiscovery = allowServerPortDiscovery;
+        this.storagePort = storagePort;
     }
 
     public void init(String keyspace)
     {
-        Cluster.Builder builder = Cluster.builder().addContactPoints(hosts).withPort(port);
+        Set<InetAddress> hostAddresses = hosts.stream().map(host -> host.getAddress()).collect(Collectors.toSet());
+        Cluster.Builder builder = Cluster.builder().addContactPoints(hostAddresses).withPort(port).allowBetaProtocolVersion();
+
+        if (allowServerPortDiscovery)
+            builder = builder.allowServerPortDiscovery();
+
         if (sslOptions != null)
             builder.withSSL(sslOptions);
         if (authProvider != null)
@@ -82,7 +99,18 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
                 Range<Token> range = new Range<>(tokenFactory.fromString(tokenRange.getStart().getValue().toString()),
                                                  tokenFactory.fromString(tokenRange.getEnd().getValue().toString()));
                 for (Host endpoint : endpoints)
-                    addRangeForEndpoint(range, endpoint.getAddress());
+                {
+                    int portToUse;
+                    if (allowServerPortDiscovery)
+                    {
+                        portToUse = endpoint.getBroadcastAddressOptPort().portOrElse(storagePort);
+                    }
+                    else
+                    {
+                        portToUse = storagePort;
+                    }
+                    addRangeForEndpoint(range, InetAddressAndPort.getByNameOverrideDefaults(endpoint.getAddress().getHostAddress(), portToUse));
+                }
             }
 
             Types types = fetchTypes(keyspace, session);
@@ -91,6 +119,10 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
             // We only need the TableMetadata for the views, so we only load that.
             tables.putAll(fetchViews(keyspace, session, partitioner, types));
         }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
     }
 
     public TableMetadataRef getTableMetadata(String tableName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 94b5832..19a0f83 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -356,7 +356,7 @@ public class UUIDGen
         * The spec says that one option is to take as many source that identify
         * this node as possible and hash them together. That's what we do here by
         * gathering all the ip of this host.
-        * Note that FBUtilities.getBroadcastAddress() should be enough to uniquely
+        * Note that FBUtilities.getJustBroadcastAddress() should be enough to uniquely
         * identify the node *in the cluster* but it triggers DatabaseDescriptor
         * instanciation and the UUID generator is used in Stress for instance,
         * where we don't want to require the yaml.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml
index ead2a88..640f9b3 100644
--- a/test/conf/cassandra.yaml
+++ b/test/conf/cassandra.yaml
@@ -17,6 +17,7 @@ hints_directory: build/test/cassandra/hints
 partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
 listen_address: 127.0.0.1
 storage_port: 7010
+ssl_storage_port: 7011
 start_native_transport: true
 native_transport_port: 9042
 column_index_size_in_kb: 4
@@ -27,7 +28,7 @@ disk_access_mode: mmap
 seed_provider:
     - class_name: org.apache.cassandra.locator.SimpleSeedProvider
       parameters:
-          - seeds: "127.0.0.1"
+          - seeds: "127.0.0.1:7010"
 endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
 dynamic_snitch: true
 server_encryption_options:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/data/serialization/4.0/gms.EndpointState.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/4.0/gms.EndpointState.bin b/test/data/serialization/4.0/gms.EndpointState.bin
index fb7d168..17fc088 100644
Binary files a/test/data/serialization/4.0/gms.EndpointState.bin and b/test/data/serialization/4.0/gms.EndpointState.bin differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/data/serialization/4.0/gms.Gossip.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/4.0/gms.Gossip.bin b/test/data/serialization/4.0/gms.Gossip.bin
index af5ac57..2fbd5d4 100644
Binary files a/test/data/serialization/4.0/gms.Gossip.bin and b/test/data/serialization/4.0/gms.Gossip.bin differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/data/serialization/4.0/service.SyncComplete.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/4.0/service.SyncComplete.bin b/test/data/serialization/4.0/service.SyncComplete.bin
index ba84349..15cccb8 100644
Binary files a/test/data/serialization/4.0/service.SyncComplete.bin and b/test/data/serialization/4.0/service.SyncComplete.bin differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/data/serialization/4.0/service.SyncRequest.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/4.0/service.SyncRequest.bin b/test/data/serialization/4.0/service.SyncRequest.bin
index 6d688a4..f4eb532 100644
Binary files a/test/data/serialization/4.0/service.SyncRequest.bin and b/test/data/serialization/4.0/service.SyncRequest.bin differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/data/serialization/4.0/service.ValidationComplete.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/4.0/service.ValidationComplete.bin b/test/data/serialization/4.0/service.ValidationComplete.bin
index 7433d64..edc90b3 100644
Binary files a/test/data/serialization/4.0/service.ValidationComplete.bin and b/test/data/serialization/4.0/service.ValidationComplete.bin differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/data/serialization/4.0/service.ValidationRequest.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/4.0/service.ValidationRequest.bin b/test/data/serialization/4.0/service.ValidationRequest.bin
index a00763b..e45eb70 100644
Binary files a/test/data/serialization/4.0/service.ValidationRequest.bin and b/test/data/serialization/4.0/service.ValidationRequest.bin differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
index 35bf5b4..a5025a3 100644
--- a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
+++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
@@ -20,7 +20,6 @@
 package org.apache.cassandra.locator;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.*;
 
 import org.junit.Test;
@@ -53,19 +52,19 @@ public class DynamicEndpointSnitchLongTest
             StorageService.instance.unsafeInitialize();
             SimpleSnitch ss = new SimpleSnitch();
             DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
-            InetAddress self = FBUtilities.getBroadcastAddress();
+            InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
 
-            List<InetAddress> hosts = new ArrayList<>();
+            List<InetAddressAndPort> hosts = new ArrayList<>();
             // We want a big list of hosts so  sorting takes time, making it much more likely to reproduce the
             // problem we're looking for.
             for (int i = 0; i < 100; i++)
                 for (int j = 0; j < 256; j++)
-                    hosts.add(InetAddress.getByAddress(new byte[]{127, 0, (byte)i, (byte)j}));
+                    hosts.add(InetAddressAndPort.getByAddress(new byte[]{ 127, 0, (byte)i, (byte)j}));
 
             ScoreUpdater updater = new ScoreUpdater(dsnitch, hosts);
             updater.start();
 
-            List<InetAddress> result = null;
+            List<InetAddressAndPort> result = null;
             for (int i = 0; i < ITERATIONS; i++)
                 result = dsnitch.getSortedListByProximity(self, hosts);
 
@@ -85,10 +84,10 @@ public class DynamicEndpointSnitchLongTest
         public volatile boolean stopped;
 
         private final DynamicEndpointSnitch dsnitch;
-        private final List<InetAddress> hosts;
+        private final List<InetAddressAndPort> hosts;
         private final Random random = new Random();
 
-        public ScoreUpdater(DynamicEndpointSnitch dsnitch, List<InetAddress> hosts)
+        public ScoreUpdater(DynamicEndpointSnitch dsnitch, List<InetAddressAndPort> hosts)
         {
             this.dsnitch = dsnitch;
             this.hosts = hosts;
@@ -98,7 +97,7 @@ public class DynamicEndpointSnitchLongTest
         {
             while (!stopped)
             {
-                InetAddress host = hosts.get(random.nextInt(hosts.size()));
+                InetAddressAndPort host = hosts.get(random.nextInt(hosts.size()));
                 int score = random.nextInt(SCORE_RANGE);
                 dsnitch.receiveTiming(host, score);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 799ac77..bd7ef20 100644
--- a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
+++ b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
@@ -122,7 +122,7 @@ public class LongStreamingTest
             public void init(String keyspace)
             {
                 for (Range<Token> range : StorageService.instance.getLocalRanges(KS))
-                    addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
+                    addRangeForEndpoint(range, FBUtilities.getBroadcastAddressAndPort());
 
                 this.ks = keyspace;
             }
@@ -149,7 +149,7 @@ public class LongStreamingTest
             public void init(String keyspace)
             {
                 for (Range<Token> range : StorageService.instance.getLocalRanges(KS))
-                    addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
+                    addRangeForEndpoint(range, FBUtilities.getBroadcastAddressAndPort());
 
                 this.ks = keyspace;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
----------------------------------------------------------------------
diff --git a/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java b/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
index 9ec1aa6..68cfd7e 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
@@ -25,11 +25,11 @@ import com.google.common.collect.Multimap;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.PendingRangeMaps;
 import org.openjdk.jmh.annotations.*;
 import org.openjdk.jmh.infra.Blackhole;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.HashSet;
@@ -50,7 +50,7 @@ public class PendingRangesBench
     PendingRangeMaps pendingRangeMaps;
     int maxToken = 256 * 100;
 
-    Multimap<Range<Token>, InetAddress> oldPendingRanges;
+    Multimap<Range<Token>, InetAddressAndPort> oldPendingRanges;
 
     private Range<Token> genRange(String left, String right)
     {
@@ -63,7 +63,7 @@ public class PendingRangesBench
         pendingRangeMaps = new PendingRangeMaps();
         oldPendingRanges = HashMultimap.create();
 
-        InetAddress[] addresses = {InetAddress.getByName("127.0.0.1"), InetAddress.getByName("127.0.0.2")};
+        InetAddressAndPort[] addresses = { InetAddressAndPort.getByName("127.0.0.1"), InetAddressAndPort.getByName("127.0.0.2")};
 
         for (int i = 0; i < maxToken; i++)
         {
@@ -97,8 +97,8 @@ public class PendingRangesBench
     {
         int randomToken = ThreadLocalRandom.current().nextInt(maxToken * 10 + 5);
         Token searchToken = new RandomPartitioner.BigIntegerToken(Integer.toString(randomToken));
-        Set<InetAddress> endpoints = new HashSet<>();
-        for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : oldPendingRanges.asMap().entrySet())
+        Set<InetAddressAndPort> endpoints = new HashSet<>();
+        for (Map.Entry<Range<Token>, Collection<InetAddressAndPort>> entry : oldPendingRanges.asMap().entrySet())
         {
             if (entry.getKey().contains(searchToken))
                 endpoints.addAll(entry.getValue());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java b/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java
index 0e4a3cf..d06caba 100644
--- a/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java
+++ b/test/unit/org/apache/cassandra/OffsetAwareConfigurationLoader.java
@@ -20,8 +20,15 @@ package org.apache.cassandra;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.YamlConfigurationLoader;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 import java.io.File;
+import java.net.Inet6Address;
+import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Joiner;
 
 
 public class OffsetAwareConfigurationLoader extends YamlConfigurationLoader
@@ -52,6 +59,31 @@ public class OffsetAwareConfigurationLoader extends YamlConfigurationLoader
         config.native_transport_port += offset;
         config.storage_port += offset;
 
+        //Rewrite the seed ports string
+        String[] hosts = config.seed_provider.parameters.get("seeds").split(",", -1);
+        String rewrittenSeeds = Joiner.on(", ").join(Arrays.stream(hosts).map(host -> {
+            StringBuilder sb = new StringBuilder();
+            try
+            {
+                InetAddressAndPort address = InetAddressAndPort.getByName(host.trim());
+                if (address.address instanceof Inet6Address)
+                {
+                     sb.append('[').append(address.address.getHostAddress()).append(']');
+                }
+                else
+                {
+                    sb.append(address.address.getHostAddress());
+                }
+                sb.append(':').append(address.port + offset);
+                return sb.toString();
+            }
+            catch (UnknownHostException e)
+            {
+                throw new ConfigurationException("Error in OffsetAwareConfigurationLoader reworking seed list", e);
+            }
+        }).collect(Collectors.toList()));
+        config.seed_provider.parameters.put("seeds", rewrittenSeeds);
+
         config.commitlog_directory += sep + offset;
         config.saved_caches_directory += sep + offset;
         config.hints_directory += sep + offset;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 7e62c41..1201efa 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -22,7 +22,6 @@ package org.apache.cassandra;
 import java.io.Closeable;
 import java.io.EOFException;
 import java.io.IOError;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -39,6 +38,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
@@ -192,7 +192,7 @@ public class Util
      * Creates initial set of nodes and tokens. Nodes are added to StorageService as 'normal'
      */
     public static void createInitialRing(StorageService ss, IPartitioner partitioner, List<Token> endpointTokens,
-                                   List<Token> keyTokens, List<InetAddress> hosts, List<UUID> hostIds, int howMany)
+                                         List<Token> keyTokens, List<InetAddressAndPort> hosts, List<UUID> hostIds, int howMany)
         throws UnknownHostException
     {
         // Expand pool of host IDs as necessary
@@ -210,10 +210,13 @@ public class Util
 
         for (int i=0; i<endpointTokens.size(); i++)
         {
-            InetAddress ep = InetAddress.getByName("127.0.0." + String.valueOf(i + 1));
+            InetAddressAndPort ep = InetAddressAndPort.getByName("127.0.0." + String.valueOf(i + 1));
             Gossiper.instance.initializeNodeUnsafe(ep, hostIds.get(i), 1);
             Gossiper.instance.injectApplicationState(ep, ApplicationState.TOKENS, new VersionedValue.VersionedValueFactory(partitioner).tokens(Collections.singleton(endpointTokens.get(i))));
             ss.onChange(ep,
+                        ApplicationState.STATUS_WITH_PORT,
+                        new VersionedValue.VersionedValueFactory(partitioner).normal(Collections.singleton(endpointTokens.get(i))));
+            ss.onChange(ep,
                         ApplicationState.STATUS,
                         new VersionedValue.VersionedValueFactory(partitioner).normal(Collections.singleton(endpointTokens.get(i))));
             hosts.add(ep);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
index 7db1cfa..41564d9 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
@@ -17,11 +17,9 @@
  */
 package org.apache.cassandra.batchlog;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 
 import com.google.common.collect.ImmutableMultimap;
@@ -29,6 +27,8 @@ import com.google.common.collect.Multimap;
 import org.junit.Test;
 import org.junit.matchers.JUnitMatchers;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
@@ -39,89 +39,89 @@ public class BatchlogEndpointFilterTest
     @Test
     public void shouldSelect2hostsFromNonLocalRacks() throws UnknownHostException
     {
-        Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
-                .put(LOCAL, InetAddress.getByName("0"))
-                .put(LOCAL, InetAddress.getByName("00"))
-                .put("1", InetAddress.getByName("1"))
-                .put("1", InetAddress.getByName("11"))
-                .put("2", InetAddress.getByName("2"))
-                .put("2", InetAddress.getByName("22"))
+        Multimap<String, InetAddressAndPort> endpoints = ImmutableMultimap.<String, InetAddressAndPort> builder()
+                .put(LOCAL, InetAddressAndPort.getByName("0"))
+                .put(LOCAL, InetAddressAndPort.getByName("00"))
+                .put("1", InetAddressAndPort.getByName("1"))
+                .put("1", InetAddressAndPort.getByName("11"))
+                .put("2", InetAddressAndPort.getByName("2"))
+                .put("2", InetAddressAndPort.getByName("22"))
                 .build();
-        Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
+        Collection<InetAddressAndPort> result = new TestEndpointFilter(LOCAL, endpoints).filter();
         assertThat(result.size(), is(2));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("11")));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("22")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("11")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("22")));
     }
 
     @Test
     public void shouldSelectHostFromLocal() throws UnknownHostException
     {
-        Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
-                .put(LOCAL, InetAddress.getByName("0"))
-                .put(LOCAL, InetAddress.getByName("00"))
-                .put("1", InetAddress.getByName("1"))
+        Multimap<String, InetAddressAndPort> endpoints = ImmutableMultimap.<String, InetAddressAndPort> builder()
+                .put(LOCAL, InetAddressAndPort.getByName("0"))
+                .put(LOCAL, InetAddressAndPort.getByName("00"))
+                .put("1", InetAddressAndPort.getByName("1"))
                 .build();
-        Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
+        Collection<InetAddressAndPort> result = new TestEndpointFilter(LOCAL, endpoints).filter();
         assertThat(result.size(), is(2));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("1")));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("0")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("1")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("0")));
     }
 
     @Test
     public void shouldReturnAsIsIfNoEnoughEndpoints() throws UnknownHostException
     {
-        Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
-                .put(LOCAL, InetAddress.getByName("0"))
+        Multimap<String, InetAddressAndPort> endpoints = ImmutableMultimap.<String, InetAddressAndPort> builder()
+                .put(LOCAL, InetAddressAndPort.getByName("0"))
                 .build();
-        Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
+        Collection<InetAddressAndPort> result = new TestEndpointFilter(LOCAL, endpoints).filter();
         assertThat(result.size(), is(1));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("0")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("0")));
     }
 
     @Test
     public void shouldSelectTwoRandomHostsFromSingleOtherRack() throws UnknownHostException
     {
-        Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
-                .put(LOCAL, InetAddress.getByName("0"))
-                .put(LOCAL, InetAddress.getByName("00"))
-                .put("1", InetAddress.getByName("1"))
-                .put("1", InetAddress.getByName("11"))
-                .put("1", InetAddress.getByName("111"))
+        Multimap<String, InetAddressAndPort> endpoints = ImmutableMultimap.<String, InetAddressAndPort> builder()
+                .put(LOCAL, InetAddressAndPort.getByName("0"))
+                .put(LOCAL, InetAddressAndPort.getByName("00"))
+                .put("1", InetAddressAndPort.getByName("1"))
+                .put("1", InetAddressAndPort.getByName("11"))
+                .put("1", InetAddressAndPort.getByName("111"))
                 .build();
-        Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
+        Collection<InetAddressAndPort> result = new TestEndpointFilter(LOCAL, endpoints).filter();
         // result should be the last two non-local replicas
         // (Collections.shuffle has been replaced with Collections.reverse for testing)
         assertThat(result.size(), is(2));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("11")));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("111")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("11")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("111")));
     }
 
     @Test
     public void shouldSelectTwoRandomHostsFromSingleRack() throws UnknownHostException
     {
-        Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
-                .put(LOCAL, InetAddress.getByName("1"))
-                .put(LOCAL, InetAddress.getByName("11"))
-                .put(LOCAL, InetAddress.getByName("111"))
-                .put(LOCAL, InetAddress.getByName("1111"))
+        Multimap<String, InetAddressAndPort> endpoints = ImmutableMultimap.<String, InetAddressAndPort> builder()
+                .put(LOCAL, InetAddressAndPort.getByName("1"))
+                .put(LOCAL, InetAddressAndPort.getByName("11"))
+                .put(LOCAL, InetAddressAndPort.getByName("111"))
+                .put(LOCAL, InetAddressAndPort.getByName("1111"))
                 .build();
-        Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
+        Collection<InetAddressAndPort> result = new TestEndpointFilter(LOCAL, endpoints).filter();
         // result should be the last two non-local replicas
         // (Collections.shuffle has been replaced with Collections.reverse for testing)
         assertThat(result.size(), is(2));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("111")));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("1111")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("111")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddressAndPort.getByName("1111")));
     }
 
     private static class TestEndpointFilter extends BatchlogManager.EndpointFilter
     {
-        TestEndpointFilter(String localRack, Multimap<String, InetAddress> endpoints)
+        TestEndpointFilter(String localRack, Multimap<String, InetAddressAndPort> endpoints)
         {
             super(localRack, endpoints);
         }
 
         @Override
-        protected boolean isValid(InetAddress input)
+        protected boolean isValid(InetAddressAndPort input)
         {
             // We will use always alive non-localhost endpoints
             return true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
index 34902fe..33fb209 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.batchlog;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.ExecutionException;
 
@@ -29,6 +28,7 @@ import org.junit.*;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.Util.PartitionerSwitcher;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -94,7 +94,7 @@ public class BatchlogManagerTest
     public void setUp() throws Exception
     {
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
-        InetAddress localhost = InetAddress.getByName("127.0.0.1");
+        InetAddressAndPort localhost = InetAddressAndPort.getByName("127.0.0.1");
         metadata.updateNormalToken(Util.token("A"), localhost);
         metadata.updateHostId(UUIDGen.getTimeUUID(), localhost);
         Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).truncateBlocking();
@@ -344,7 +344,7 @@ public class BatchlogManagerTest
     @Test
     public void testReplayWithNoPeers() throws Exception
     {
-        StorageService.instance.getTokenMetadata().removeEndpoint(InetAddress.getByName("127.0.0.1"));
+        StorageService.instance.getTokenMetadata().removeEndpoint(InetAddressAndPort.getByName("127.0.0.1"));
 
         long initialAllBatches = BatchlogManager.instance.countAllBatches();
         long initialReplayedBatches = BatchlogManager.instance.getTotalBatchesReplayed();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
index b50a050..589afd5 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
@@ -77,6 +77,7 @@ public class DatabaseDescriptorRefTest
     "org.apache.cassandra.config.EncryptionOptions$ClientEncryptionOptions",
     "org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptions",
     "org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptions$InternodeEncryption",
+    "org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptions$OutgoingEncryptedPortSource",
     "org.apache.cassandra.config.YamlConfigurationLoader",
     "org.apache.cassandra.config.YamlConfigurationLoader$PropertiesChecker",
     "org.apache.cassandra.config.YamlConfigurationLoader$PropertiesChecker$1",
@@ -126,6 +127,7 @@ public class DatabaseDescriptorRefTest
     "org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptionsCustomizer",
     "org.apache.cassandra.ConsoleAppenderBeanInfo",
     "org.apache.cassandra.ConsoleAppenderCustomizer",
+    "org.apache.cassandra.locator.InetAddressAndPort"
     };
 
     static final Set<String> checkedClasses = new HashSet<>(Arrays.asList(validClasses));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 6993a65..69d2fb5 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -48,6 +48,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.index.SecondaryIndexManager;
 import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.functions.FunctionName;
@@ -144,9 +145,9 @@ public abstract class CQLTester
         // Register an EndpointSnitch which returns fixed values for test.
         DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
         {
-            @Override public String getRack(InetAddress endpoint) { return RACK1; }
-            @Override public String getDatacenter(InetAddress endpoint) { return DATA_CENTER; }
-            @Override public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2) { return 0; }
+            @Override public String getRack(InetAddressAndPort endpoint) { return RACK1; }
+            @Override public String getDatacenter(InetAddressAndPort endpoint) { return DATA_CENTER; }
+            @Override public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2) { return 0; }
         });
 
         try
@@ -829,6 +830,11 @@ public abstract class CQLTester
         return sessionNet(protocolVersion).execute(formatQuery(query), values);
     }
 
+    protected com.datastax.driver.core.ResultSet executeNetWithPaging(ProtocolVersion version, String query, int pageSize) throws Throwable
+    {
+        return sessionNet(version).execute(new SimpleStatement(formatQuery(query)).setFetchSize(pageSize));
+    }
+
     protected com.datastax.driver.core.ResultSet executeNetWithPaging(String query, int pageSize) throws Throwable
     {
         return sessionNet().execute(new SimpleStatement(formatQuery(query)).setFetchSize(pageSize));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 0a314da..ce5de62 100644
--- a/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
@@ -223,7 +223,6 @@ public class PreparedStatementsTest extends CQLTester
                                  .withClusterName("Test Cluster")
                                  .withPort(nativePort)
                                  .withoutJMXReporting()
-                                 .allowBetaProtocolVersion()
                                  .build())
         {
             try (Session newSession = newCluster.connect())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java b/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
index 26b9d65..665bc44 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
@@ -851,10 +851,10 @@ public class ViewComplexTest extends CQLTester
         for (String view : Arrays.asList("mv1", "mv2"))
         {
             // paging
-            assertEquals(1, executeNetWithPaging(String.format("SELECT k,a,b FROM %s limit 1", view), 1).all().size());
-            assertEquals(2, executeNetWithPaging(String.format("SELECT k,a,b FROM %s limit 2", view), 1).all().size());
-            assertEquals(2, executeNetWithPaging(String.format("SELECT k,a,b FROM %s", view), 1).all().size());
-            assertRowsNet(executeNetWithPaging(String.format("SELECT k,a,b FROM %s ", view), 1),
+            assertEquals(1, executeNetWithPaging(protocolVersion, String.format("SELECT k,a,b FROM %s limit 1", view), 1).all().size());
+            assertEquals(2, executeNetWithPaging(protocolVersion, String.format("SELECT k,a,b FROM %s limit 2", view), 1).all().size());
+            assertEquals(2, executeNetWithPaging(protocolVersion, String.format("SELECT k,a,b FROM %s", view), 1).all().size());
+            assertRowsNet(protocolVersion, executeNetWithPaging(protocolVersion, String.format("SELECT k,a,b FROM %s ", view), 1),
                           row(50, 50, 50),
                           row(100, 100, 100));
             // limit

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
index ac261ca..dc90b4e 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.cql3.validation.operations;
 
-import java.net.InetAddress;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collection;
@@ -26,6 +25,7 @@ import java.util.UUID;
 
 import org.junit.Test;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -514,13 +514,13 @@ public class CreateTest extends CQLTester
         DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
         {
             @Override
-            public String getRack(InetAddress endpoint) { return RACK1; }
+            public String getRack(InetAddressAndPort endpoint) { return RACK1; }
 
             @Override
-            public String getDatacenter(InetAddress endpoint) { return "us-east-1"; }
+            public String getDatacenter(InetAddressAndPort endpoint) { return "us-east-1"; }
 
             @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2) { return 0; }
+            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2) { return 0; }
         });
 
         execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east-1' : 1 };");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 044a49e..a096c78 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.AbstractMap;
@@ -37,6 +36,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.KeyspaceMetadata;
@@ -89,13 +89,13 @@ public class CleanupTest
         DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch()
         {
             @Override
-            public String getRack(InetAddress endpoint)
+            public String getRack(InetAddressAndPort endpoint)
             {
                 return "RC1";
             }
 
             @Override
-            public String getDatacenter(InetAddress endpoint)
+            public String getDatacenter(InetAddressAndPort endpoint)
             {
                 return "DC1";
             }
@@ -166,8 +166,8 @@ public class CleanupTest
         byte[] tk1 = new byte[1], tk2 = new byte[1];
         tk1[0] = 2;
         tk2[0] = 1;
-        tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
-        tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
+        tmd.updateNormalToken(new BytesToken(tk1), InetAddressAndPort.getByName("127.0.0.1"));
+        tmd.updateNormalToken(new BytesToken(tk2), InetAddressAndPort.getByName("127.0.0.2"));
 
         CompactionManager.instance.performCleanup(cfs, 2);
 
@@ -198,8 +198,8 @@ public class CleanupTest
         byte[] tk1 = new byte[1], tk2 = new byte[1];
         tk1[0] = 2;
         tk2[0] = 1;
-        tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
-        tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
+        tmd.updateNormalToken(new BytesToken(tk1), InetAddressAndPort.getByName("127.0.0.1"));
+        tmd.updateNormalToken(new BytesToken(tk2), InetAddressAndPort.getByName("127.0.0.2"));
         CompactionManager.instance.performCleanup(cfs, 2);
 
         assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
@@ -222,9 +222,9 @@ public class CleanupTest
 
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
         tmd.clearUnsafe();
-        tmd.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.0.0.1"));
+        tmd.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.0.0.1"));
         byte[] tk1 = {2};
-        tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
+        tmd.updateNormalToken(new BytesToken(tk1), InetAddressAndPort.getByName("127.0.0.1"));
 
 
         Keyspace keyspace = Keyspace.open(KEYSPACE2);
@@ -270,8 +270,8 @@ public class CleanupTest
         byte[] tk1 = new byte[1], tk2 = new byte[1];
         tk1[0] = 2;
         tk2[0] = 1;
-        tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
-        tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
+        tmd.updateNormalToken(new BytesToken(tk1), InetAddressAndPort.getByName("127.0.0.1"));
+        tmd.updateNormalToken(new BytesToken(tk2), InetAddressAndPort.getByName("127.0.0.2"));
 
         for(SSTableReader r: cfs.getLiveSSTables())
             CompactionManager.instance.forceUserDefinedCleanup(r.getFilename());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/DiskBoundaryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DiskBoundaryManagerTest.java b/test/unit/org/apache/cassandra/db/DiskBoundaryManagerTest.java
index c56368f..6e2a714 100644
--- a/test/unit/org/apache/cassandra/db/DiskBoundaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/DiskBoundaryManagerTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.db;
 
 import java.io.File;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.List;
 
@@ -30,6 +29,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.dht.BootStrapper;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
@@ -51,7 +51,7 @@ public class DiskBoundaryManagerTest extends CQLTester
     {
         BlacklistedDirectories.clearUnwritableUnsafe();
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
-        metadata.updateNormalTokens(BootStrapper.getRandomTokens(metadata, 10), FBUtilities.getBroadcastAddress());
+        metadata.updateNormalTokens(BootStrapper.getRandomTokens(metadata, 10), FBUtilities.getBroadcastAddressAndPort());
         createTable("create table %s (id int primary key, x text)");
         dirs = new Directories(getCurrentColumnFamilyStore().metadata(), Lists.newArrayList(new Directories.DataDirectory(new File("/tmp/1")),
                                                                                           new Directories.DataDirectory(new File("/tmp/2")),
@@ -86,7 +86,7 @@ public class DiskBoundaryManagerTest extends CQLTester
     public void updateTokensTest() throws UnknownHostException
     {
         DiskBoundaries dbv1 = dbm.getDiskBoundaries(mock);
-        StorageService.instance.getTokenMetadata().updateNormalTokens(BootStrapper.getRandomTokens(StorageService.instance.getTokenMetadata(), 10), InetAddress.getByName("127.0.0.10"));
+        StorageService.instance.getTokenMetadata().updateNormalTokens(BootStrapper.getRandomTokens(StorageService.instance.getTokenMetadata(), 10), InetAddressAndPort.getByName("127.0.0.10"));
         DiskBoundaries dbv2 = dbm.getDiskBoundaries(mock);
         assertFalse(dbv1.equals(dbv2));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 64ac627..6bb0a1a 100644
--- a/test/unit/org/apache/cassandra/db/ReadCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadCommandTest.java
@@ -50,9 +50,11 @@ 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.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -312,7 +314,8 @@ public class ReadCommandTest
         }
     }
 
-    public void serializerTest() throws IOException
+    @Test
+    public void testSerializer() throws IOException
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF2);
 
@@ -324,10 +327,11 @@ public class ReadCommandTest
 
         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);
+        Tracing.instance.newSession(Tracing.TraceType.QUERY);
+        MessageOut<ReadCommand> messageOut = new MessageOut(MessagingService.Verb.READ, readCommand, ReadCommand.serializer);
+        long size = messageOut.serializedSize(messagingVersion);
+        messageOut.serialize(new WrappedDataOutputStreamPlus(out), messagingVersion);
         Assert.assertEquals(size, out.count);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/RowCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index fee3f2c..5ca1eef 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db;
 
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -32,6 +31,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.RowCacheKey;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.rows.*;
@@ -300,8 +300,8 @@ public class RowCacheTest
         byte[] tk1, tk2;
         tk1 = "key1000".getBytes();
         tk2 = "key1050".getBytes();
-        tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
-        tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
+        tmd.updateNormalToken(new BytesToken(tk1), InetAddressAndPort.getByName("127.0.0.1"));
+        tmd.updateNormalToken(new BytesToken(tk2), InetAddressAndPort.getByName("127.0.0.2"));
         store.cleanupCache();
         assertEquals(50, CacheService.instance.rowCache.size());
         CacheService.instance.setRowCacheCapacityInMB(0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java
new file mode 100644
index 0000000..1c051f5
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java
@@ -0,0 +1,192 @@
+/*
+ * 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.db;
+
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.utils.UUIDGen;
+
+import static org.junit.Assert.assertEquals;
+
+public class SystemKeyspaceMigrator40Test extends CQLTester
+{
+    @Test
+    public void testMigratePeers() throws Throwable
+    {
+        String insert = String.format("INSERT INTO %s ("
+                                      + "peer, "
+                                      + "data_center, "
+                                      + "host_id, "
+                                      + "preferred_ip, "
+                                      + "rack, "
+                                      + "release_version, "
+                                      + "rpc_address, "
+                                      + "schema_version, "
+                                      + "tokens) "
+                                      + " values ( ?, ?, ? , ? , ?, ?, ?, ?, ?)",
+                                      SystemKeyspaceMigrator40.legacyPeersName);
+        UUID hostId = UUIDGen.getTimeUUID();
+        UUID schemaVersion = UUIDGen.getTimeUUID();
+        execute(insert,
+                InetAddress.getByName("127.0.0.1"),
+                "dcFoo",
+                hostId,
+                InetAddress.getByName("127.0.0.2"),
+                "rackFoo", "4.0",
+                InetAddress.getByName("127.0.0.3"),
+                schemaVersion,
+                ImmutableSet.of("foobar"));
+        SystemKeyspaceMigrator40.migrate();
+
+        int rowCount = 0;
+        for (UntypedResultSet.Row row : execute(String.format("SELECT * FROM %s", SystemKeyspaceMigrator40.peersName)))
+        {
+            rowCount++;
+            assertEquals(InetAddress.getByName("127.0.0.1"), row.getInetAddress("peer"));
+            assertEquals(DatabaseDescriptor.getStoragePort(), row.getInt("peer_port"));
+            assertEquals("dcFoo", row.getString("data_center"));
+            assertEquals(hostId, row.getUUID("host_id"));
+            assertEquals(InetAddress.getByName("127.0.0.2"), row.getInetAddress("preferred_ip"));
+            assertEquals(DatabaseDescriptor.getStoragePort(), row.getInt("preferred_port"));
+            assertEquals("rackFoo", row.getString("rack"));
+            assertEquals("4.0", row.getString("release_version"));
+            assertEquals(InetAddress.getByName("127.0.0.3"), row.getInetAddress("native_address"));
+            assertEquals(DatabaseDescriptor.getNativeTransportPort(), row.getInt("native_port"));
+            assertEquals(schemaVersion, row.getUUID("schema_version"));
+            assertEquals(ImmutableSet.of("foobar"), row.getSet("tokens", UTF8Type.instance));
+        }
+        assertEquals(1, rowCount);
+
+        //Test nulls/missing don't prevent the row from propagating
+        execute(String.format("TRUNCATE %s", SystemKeyspaceMigrator40.legacyPeersName));
+        execute(String.format("TRUNCATE %s", SystemKeyspaceMigrator40.peersName));
+
+        execute(String.format("INSERT INTO %s (peer) VALUES (?)", SystemKeyspaceMigrator40.legacyPeersName),
+                              InetAddress.getByName("127.0.0.1"));
+        SystemKeyspaceMigrator40.migrate();
+
+        rowCount = 0;
+        for (UntypedResultSet.Row row : execute(String.format("SELECT * FROM %s", SystemKeyspaceMigrator40.peersName)))
+        {
+            rowCount++;
+        }
+        assertEquals(1, rowCount);
+    }
+
+    @Test
+    public void testMigratePeerEvents() throws Throwable
+    {
+        String insert = String.format("INSERT INTO %s ("
+                                      + "peer, "
+                                      + "hints_dropped) "
+                                      + " values ( ?, ? )",
+                                      SystemKeyspaceMigrator40.legacyPeerEventsName);
+        UUID uuid = UUIDGen.getTimeUUID();
+        execute(insert,
+                InetAddress.getByName("127.0.0.1"),
+                ImmutableMap.of(uuid, 42));
+        SystemKeyspaceMigrator40.migrate();
+
+        int rowCount = 0;
+        for (UntypedResultSet.Row row : execute(String.format("SELECT * FROM %s", SystemKeyspaceMigrator40.peerEventsName)))
+        {
+            rowCount++;
+            assertEquals(InetAddress.getByName("127.0.0.1"), row.getInetAddress("peer"));
+            assertEquals(DatabaseDescriptor.getStoragePort(), row.getInt("peer_port"));
+            assertEquals(ImmutableMap.of(uuid, 42), row.getMap("hints_dropped", UUIDType.instance, Int32Type.instance));
+        }
+        assertEquals(1, rowCount);
+
+        //Test nulls/missing don't prevent the row from propagating
+        execute(String.format("TRUNCATE %s", SystemKeyspaceMigrator40.legacyPeerEventsName));
+        execute(String.format("TRUNCATE %s", SystemKeyspaceMigrator40.peerEventsName));
+
+        execute(String.format("INSERT INTO %s (peer) VALUES (?)", SystemKeyspaceMigrator40.legacyPeerEventsName),
+                InetAddress.getByName("127.0.0.1"));
+        SystemKeyspaceMigrator40.migrate();
+
+        rowCount = 0;
+        for (UntypedResultSet.Row row : execute(String.format("SELECT * FROM %s", SystemKeyspaceMigrator40.peerEventsName)))
+        {
+            rowCount++;
+        }
+        assertEquals(1, rowCount);
+    }
+
+    @Test
+    public void testMigrateTransferredRanges() throws Throwable
+    {
+        String insert = String.format("INSERT INTO %s ("
+                                      + "operation, "
+                                      + "peer, "
+                                      + "keyspace_name, "
+                                      + "ranges) "
+                                      + " values ( ?, ?, ?, ? )",
+                                      SystemKeyspaceMigrator40.legacyTransferredRangesName);
+        execute(insert,
+                "foo",
+                InetAddress.getByName("127.0.0.1"),
+                "bar",
+                ImmutableSet.of(ByteBuffer.wrap(new byte[] { 42 })));
+        SystemKeyspaceMigrator40.migrate();
+
+        int rowCount = 0;
+        for (UntypedResultSet.Row row : execute(String.format("SELECT * FROM %s", SystemKeyspaceMigrator40.transferredRangesName)))
+        {
+            rowCount++;
+            assertEquals("foo", row.getString("operation"));
+            assertEquals(InetAddress.getByName("127.0.0.1"), row.getInetAddress("peer"));
+            assertEquals(DatabaseDescriptor.getStoragePort(), row.getInt("peer_port"));
+            assertEquals("bar", row.getString("keyspace_name"));
+            assertEquals(ImmutableSet.of(ByteBuffer.wrap(new byte[] { 42 })), row.getSet("ranges", BytesType.instance));
+        }
+        assertEquals(1, rowCount);
+
+        //Test nulls/missing don't prevent the row from propagating
+        execute(String.format("TRUNCATE %s", SystemKeyspaceMigrator40.legacyTransferredRangesName));
+        execute(String.format("TRUNCATE %s", SystemKeyspaceMigrator40.transferredRangesName));
+
+        execute(String.format("INSERT INTO %s (operation, peer, keyspace_name) VALUES (?, ?, ?)", SystemKeyspaceMigrator40.legacyTransferredRangesName),
+                "foo",
+                InetAddress.getByName("127.0.0.1"),
+                "bar");
+        SystemKeyspaceMigrator40.migrate();
+
+        rowCount = 0;
+        for (UntypedResultSet.Row row : execute(String.format("SELECT * FROM %s", SystemKeyspaceMigrator40.transferredRangesName)))
+        {
+            rowCount++;
+        }
+        assertEquals(1, rowCount);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index d1b8ff5..3bc04c1 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 
@@ -26,6 +25,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -53,7 +53,7 @@ public class SystemKeyspaceTest
     public void testLocalTokens()
     {
         // Remove all existing tokens
-        Collection<Token> current = SystemKeyspace.loadTokens().asMap().get(FBUtilities.getLocalAddress());
+        Collection<Token> current = SystemKeyspace.loadTokens().asMap().get(FBUtilities.getLocalAddressAndPort());
         if (current != null && !current.isEmpty())
             SystemKeyspace.updateTokens(current);
 
@@ -74,7 +74,7 @@ public class SystemKeyspaceTest
     public void testNonLocalToken() throws UnknownHostException
     {
         BytesToken token = new BytesToken(ByteBufferUtil.bytes("token3"));
-        InetAddress address = InetAddress.getByName("127.0.0.2");
+        InetAddressAndPort address = InetAddressAndPort.getByName("127.0.0.2");
         SystemKeyspace.updateTokens(address, Collections.<Token>singletonList(token));
         assert SystemKeyspace.loadTokens().get(address).contains(token);
         SystemKeyspace.removeEndpoint(address);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
index 0baad3b..3e38dfc 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.db.compaction;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.UUID;
@@ -31,6 +30,7 @@ import org.junit.Ignore;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
@@ -67,7 +67,7 @@ public class AbstractPendingRepairTest extends AbstractRepairTest
         // cutoff messaging service
         MessagingService.instance().addMessageSink(new IMessageSink()
         {
-            public boolean allowOutgoingMessage(MessageOut message, int id, InetAddress to)
+            public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to)
             {
                 return false;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index 5f05fab..6e7e184 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.compaction;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -35,6 +34,7 @@ import org.junit.After;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
@@ -96,7 +96,7 @@ public class AntiCompactionTest
     private void registerParentRepairSession(UUID sessionID, Collection<Range<Token>> ranges, long repairedAt, UUID pendingRepair) throws IOException
     {
         ActiveRepairService.instance.registerParentRepairSession(sessionID,
-                                                                 InetAddress.getByName("10.0.0.1"),
+                                                                 InetAddressAndPort.getByName("10.0.0.1"),
                                                                  Lists.newArrayList(cfs), ranges,
                                                                  pendingRepair != null || repairedAt != UNREPAIRED_SSTABLE,
                                                                  repairedAt, true, PreviewKind.NONE);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/compaction/CompactionManagerGetSSTablesForValidationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionManagerGetSSTablesForValidationTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionManagerGetSSTablesForValidationTest.java
index b9e3c17..8290adf 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionManagerGetSSTablesForValidationTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionManagerGetSSTablesForValidationTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db.compaction;
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Set;
@@ -33,6 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -58,7 +58,7 @@ public class CompactionManagerGetSSTablesForValidationTest
     private String ks;
     private static final String tbl = "tbl";
     private ColumnFamilyStore cfs;
-    private static InetAddress coordinator;
+    private static InetAddressAndPort coordinator;
 
     private static Token MT;
 
@@ -73,7 +73,7 @@ public class CompactionManagerGetSSTablesForValidationTest
     public static void setupClass() throws Exception
     {
         SchemaLoader.prepareServer();
-        coordinator = InetAddress.getByName("10.0.0.1");
+        coordinator = InetAddressAndPort.getByName("10.0.0.1");
         MT = DatabaseDescriptor.getPartitioner().getMinimumToken();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 624f119..567984d 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -195,7 +195,7 @@ public class LeveledCompactionStrategyTest
         int gcBefore = keyspace.getColumnFamilyStore(CF_STANDARDDLEVELED).gcBefore(FBUtilities.nowInSeconds());
         UUID parentRepSession = UUID.randomUUID();
         ActiveRepairService.instance.registerParentRepairSession(parentRepSession,
-                                                                 FBUtilities.getBroadcastAddress(),
+                                                                 FBUtilities.getBroadcastAddressAndPort(),
                                                                  Arrays.asList(cfs),
                                                                  Arrays.asList(range),
                                                                  false,
@@ -203,7 +203,7 @@ public class LeveledCompactionStrategyTest
                                                                  true,
                                                                  PreviewKind.NONE);
         RepairJobDesc desc = new RepairJobDesc(parentRepSession, UUID.randomUUID(), KEYSPACE1, CF_STANDARDDLEVELED, Arrays.asList(range));
-        Validator validator = new Validator(desc, FBUtilities.getBroadcastAddress(), gcBefore, PreviewKind.NONE);
+        Validator validator = new Validator(desc, FBUtilities.getBroadcastAddressAndPort(), gcBefore, PreviewKind.NONE);
         CompactionManager.instance.submitValidation(cfs, validator).get();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
index 658b87a..423ad28 100644
--- a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
+++ b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db.view;
 
-import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
@@ -28,6 +27,7 @@ import org.junit.Test;
 
 import junit.framework.Assert;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
@@ -59,12 +59,12 @@ public class ViewUtilsTest
         metadata.clearUnsafe();
 
         // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.2"));
 
         // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddressAndPort.getByName("127.0.0.5"));
 
         Map<String, String> replicationMap = new HashMap<>();
         replicationMap.put(ReplicationParams.CLASS, NetworkTopologyStrategy.class.getName());
@@ -76,12 +76,12 @@ public class ViewUtilsTest
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
         Schema.instance.load(meta);
 
-        Optional<InetAddress> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
-                                                                       new StringToken("CA"),
-                                                                       new StringToken("BB"));
+        Optional<InetAddressAndPort> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
+                                                                                        new StringToken("CA"),
+                                                                                        new StringToken("BB"));
 
         Assert.assertTrue(naturalEndpoint.isPresent());
-        Assert.assertEquals(InetAddress.getByName("127.0.0.2"), naturalEndpoint.get());
+        Assert.assertEquals(InetAddressAndPort.getByName("127.0.0.2"), naturalEndpoint.get());
     }
 
 
@@ -92,12 +92,12 @@ public class ViewUtilsTest
         metadata.clearUnsafe();
 
         // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.2"));
 
         // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddressAndPort.getByName("127.0.0.5"));
 
         Map<String, String> replicationMap = new HashMap<>();
         replicationMap.put(ReplicationParams.CLASS, NetworkTopologyStrategy.class.getName());
@@ -109,12 +109,12 @@ public class ViewUtilsTest
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
         Schema.instance.load(meta);
 
-        Optional<InetAddress> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
-                                                                       new StringToken("CA"),
-                                                                       new StringToken("BB"));
+        Optional<InetAddressAndPort> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
+                                                                                        new StringToken("CA"),
+                                                                                        new StringToken("BB"));
 
         Assert.assertTrue(naturalEndpoint.isPresent());
-        Assert.assertEquals(InetAddress.getByName("127.0.0.1"), naturalEndpoint.get());
+        Assert.assertEquals(InetAddressAndPort.getByName("127.0.0.1"), naturalEndpoint.get());
     }
 
     @Test
@@ -124,12 +124,12 @@ public class ViewUtilsTest
         metadata.clearUnsafe();
 
         // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+        metadata.updateNormalToken(new StringToken("A"), InetAddressAndPort.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddressAndPort.getByName("127.0.0.2"));
 
         // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+        metadata.updateNormalToken(new StringToken("B"), InetAddressAndPort.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddressAndPort.getByName("127.0.0.5"));
 
         Map<String, String> replicationMap = new HashMap<>();
         replicationMap.put(ReplicationParams.CLASS, NetworkTopologyStrategy.class.getName());
@@ -141,9 +141,9 @@ public class ViewUtilsTest
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
         Schema.instance.load(meta);
 
-        Optional<InetAddress> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
-                                                                       new StringToken("AB"),
-                                                                       new StringToken("BB"));
+        Optional<InetAddressAndPort> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
+                                                                                        new StringToken("AB"),
+                                                                                        new StringToken("BB"));
 
         Assert.assertFalse(naturalEndpoint.isPresent());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index a1054bb..f11cb62 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.dht;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.HashSet;
@@ -41,6 +40,7 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.tokenallocator.TokenAllocation;
@@ -96,32 +96,32 @@ public class BootStrapperTest
 
         generateFakeEndpoints(numOldNodes);
         Token myToken = tmd.partitioner.getRandomToken();
-        InetAddress myEndpoint = InetAddress.getByName("127.0.0.1");
+        InetAddressAndPort myEndpoint = InetAddressAndPort.getByName("127.0.0.1");
 
         assertEquals(numOldNodes, tmd.sortedTokens().size());
         RangeStreamer s = new RangeStreamer(tmd, null, myEndpoint, StreamOperation.BOOTSTRAP, true, DatabaseDescriptor.getEndpointSnitch(), new StreamStateStore(), false, 1);
         IFailureDetector mockFailureDetector = new IFailureDetector()
         {
-            public boolean isAlive(InetAddress ep)
+            public boolean isAlive(InetAddressAndPort ep)
             {
                 return true;
             }
 
-            public void interpret(InetAddress ep) { throw new UnsupportedOperationException(); }
-            public void report(InetAddress ep) { throw new UnsupportedOperationException(); }
+            public void interpret(InetAddressAndPort ep) { throw new UnsupportedOperationException(); }
+            public void report(InetAddressAndPort ep) { throw new UnsupportedOperationException(); }
             public void registerFailureDetectionEventListener(IFailureDetectionEventListener listener) { throw new UnsupportedOperationException(); }
             public void unregisterFailureDetectionEventListener(IFailureDetectionEventListener listener) { throw new UnsupportedOperationException(); }
-            public void remove(InetAddress ep) { throw new UnsupportedOperationException(); }
-            public void forceConviction(InetAddress ep) { throw new UnsupportedOperationException(); }
+            public void remove(InetAddressAndPort ep) { throw new UnsupportedOperationException(); }
+            public void forceConviction(InetAddressAndPort ep) { throw new UnsupportedOperationException(); }
         };
         s.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(mockFailureDetector));
         s.addRanges(keyspaceName, Keyspace.open(keyspaceName).getReplicationStrategy().getPendingAddressRanges(tmd, myToken, myEndpoint));
 
-        Collection<Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = s.toFetch().get(keyspaceName);
+        Collection<Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> toFetch = s.toFetch().get(keyspaceName);
 
         // Check we get get RF new ranges in total
         Set<Range<Token>> ranges = new HashSet<>();
-        for (Map.Entry<InetAddress, Collection<Range<Token>>> e : toFetch)
+        for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> e : toFetch)
             ranges.addAll(e.getValue());
 
         assertEquals(replicationFactor, ranges.size());
@@ -151,7 +151,7 @@ public class BootStrapperTest
         for (int i = 1; i <= numOldNodes; i++)
         {
             // leave .1 for myEndpoint
-            InetAddress addr = InetAddress.getByName("127." + dc + "." + rack + "." + (i + 1));
+            InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc + "." + rack + "." + (i + 1));
             List<Token> tokens = Lists.newArrayListWithCapacity(numVNodes);
             for (int j = 0; j < numVNodes; ++j)
                 tokens.add(p.getRandomToken());
@@ -167,7 +167,7 @@ public class BootStrapperTest
         String ks = "BootStrapperTestKeyspace3";
         TokenMetadata tm = new TokenMetadata();
         generateFakeEndpoints(tm, 10, vn);
-        InetAddress addr = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort addr = FBUtilities.getBroadcastAddressAndPort();
         allocateTokensForNode(vn, ks, tm, addr);
     }
 
@@ -184,15 +184,15 @@ public class BootStrapperTest
             // Register peers with expected DC for NetworkTopologyStrategy.
             TokenMetadata metadata = StorageService.instance.getTokenMetadata();
             metadata.clearUnsafe();
-            metadata.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.1.0.99"));
-            metadata.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.15.0.99"));
+            metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.1.0.99"));
+            metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.15.0.99"));
 
             SchemaLoader.createKeyspace(ks, KeyspaceParams.nts(dc, replicas, "15", 15), SchemaLoader.standardCFMD(ks, "Standard1"));
             TokenMetadata tm = StorageService.instance.getTokenMetadata();
             tm.clearUnsafe();
             for (int i = 0; i < rackCount; ++i)
                 generateFakeEndpoints(tm, 10, vn, dc, Integer.toString(i));
-            InetAddress addr = InetAddress.getByName("127." + dc + ".0.99");
+            InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc + ".0.99");
             allocateTokensForNode(vn, ks, tm, addr);
             // Note: Not matching replication factor in second datacentre, but this should not affect us.
         } finally {
@@ -230,7 +230,7 @@ public class BootStrapperTest
         testAllocateTokensNetworkStrategy(1, 1);
     }
 
-    private void allocateTokensForNode(int vn, String ks, TokenMetadata tm, InetAddress addr)
+    private void allocateTokensForNode(int vn, String ks, TokenMetadata tm, InetAddressAndPort addr)
     {
         SummaryStatistics os = TokenAllocation.replicatedOwnershipStats(tm.cloneOnlyTokenMap(), Keyspace.open(ks).getReplicationStrategy(), addr);
         Collection<Token> tokens = BootStrapper.allocateTokens(tm, addr, ks, vn, 0);
@@ -260,14 +260,14 @@ public class BootStrapperTest
         TokenMetadata tm = new TokenMetadata();
         generateFakeEndpoints(tm, 10, vn);
         
-        InetAddress dcaddr = FBUtilities.getBroadcastAddress();
+        InetAddressAndPort dcaddr = FBUtilities.getBroadcastAddressAndPort();
         SummaryStatistics os3 = TokenAllocation.replicatedOwnershipStats(tm, Keyspace.open(ks3).getReplicationStrategy(), dcaddr);
         SummaryStatistics os2 = TokenAllocation.replicatedOwnershipStats(tm, Keyspace.open(ks2).getReplicationStrategy(), dcaddr);
         String cks = ks3;
         String nks = ks2;
         for (int i=11; i<=20; ++i)
         {
-            allocateTokensForNode(vn, cks, tm, InetAddress.getByName("127.0.0." + (i + 1)));
+            allocateTokensForNode(vn, cks, tm, InetAddressAndPort.getByName("127.0.0." + (i + 1)));
             String t = cks; cks = nks; nks = t;
         }
         


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


[19/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
Allow storage port to be configurable per node

Patch by Ariel Weisberg; Reviewed by Jason Brown for CASSANDRA-7544


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

Branch: refs/heads/trunk
Commit: 59b5b6bef0fa76bf5740b688fcd4d9cf525760d0
Parents: 4de7a65
Author: Ariel Weisberg <aw...@apple.com>
Authored: Thu Nov 9 11:33:48 2017 -0500
Committer: Ariel Weisberg <aw...@apple.com>
Committed: Thu Jan 25 14:32:24 2018 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  18 +-
 bin/cqlsh.py                                    |  12 +-
 build.xml                                       |  10 +-
 conf/cassandra.yaml                             |   9 +-
 ...iver-core-3.3.2-0461ed35-SNAPSHOT-shaded.jar | Bin 2618371 -> 0 bytes
 ...sandra-driver-core-4.0.0-SNAPSHOT-shaded.jar | Bin 0 -> 2621460 bytes
 lib/cassandra-driver-internal-only-3.11.zip     | Bin 264882 -> 0 bytes
 ...iver-internal-only-3.12.0.post0-9ee88ded.zip | Bin 0 -> 265110 bytes
 .../cassandra/batchlog/BatchlogManager.java     |  52 +-
 .../cassandra/config/DatabaseDescriptor.java    |  27 +-
 .../apache/cassandra/db/ConsistencyLevel.java   |  40 +-
 .../db/CounterMutationVerbHandler.java          |   2 +-
 .../cassandra/db/DiskBoundaryManager.java       |   4 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   1 +
 src/java/org/apache/cassandra/db/Mutation.java  |   3 -
 .../cassandra/db/MutationVerbHandler.java       |  43 +-
 .../cassandra/db/SizeEstimatesRecorder.java     |   4 +-
 .../org/apache/cassandra/db/SystemKeyspace.java | 522 +++++++-----
 .../cassandra/db/SystemKeyspaceMigrator40.java  | 184 +++++
 .../org/apache/cassandra/db/view/ViewUtils.java |  18 +-
 .../org/apache/cassandra/dht/BootStrapper.java  |  12 +-
 .../cassandra/dht/RangeFetchMapCalculator.java  |  37 +-
 .../org/apache/cassandra/dht/RangeStreamer.java |  96 +--
 .../dht/tokenallocator/TokenAllocation.java     |  50 +-
 .../tokenallocator/TokenAllocatorFactory.java   |   8 +-
 .../exceptions/ReadFailureException.java        |   4 +-
 .../exceptions/RequestFailureException.java     |   6 +-
 .../exceptions/WriteFailureException.java       |   4 +-
 .../apache/cassandra/gms/ApplicationState.java  |  11 +-
 .../org/apache/cassandra/gms/EndpointState.java |   8 +-
 .../apache/cassandra/gms/FailureDetector.java   |  74 +-
 .../cassandra/gms/FailureDetectorMBean.java     |   9 +-
 .../org/apache/cassandra/gms/GossipDigest.java  |  14 +-
 .../apache/cassandra/gms/GossipDigestAck.java   |  22 +-
 .../apache/cassandra/gms/GossipDigestAck2.java  |  22 +-
 .../gms/GossipDigestAck2VerbHandler.java        |   6 +-
 .../gms/GossipDigestAckVerbHandler.java         |  10 +-
 .../gms/GossipDigestSynVerbHandler.java         |  12 +-
 src/java/org/apache/cassandra/gms/Gossiper.java | 274 ++++---
 .../gms/IEndpointStateChangeSubscriber.java     |  16 +-
 .../gms/IFailureDetectionEventListener.java     |   4 +-
 .../apache/cassandra/gms/IFailureDetector.java  |  12 +-
 .../apache/cassandra/gms/VersionedValue.java    |  17 +
 .../apache/cassandra/hadoop/ConfigHelper.java   |  12 +
 .../hadoop/cql3/CqlBulkRecordWriter.java        |  20 +-
 .../cassandra/hadoop/cql3/CqlConfigHelper.java  |  12 +
 .../apache/cassandra/hints/HintVerbHandler.java |   6 +-
 .../cassandra/hints/HintsDispatchExecutor.java  |  12 +-
 .../cassandra/hints/HintsDispatchTrigger.java   |   2 +-
 .../apache/cassandra/hints/HintsDispatcher.java |  10 +-
 .../apache/cassandra/hints/HintsService.java    |   8 +-
 .../org/apache/cassandra/hints/HintsStore.java  |   6 +-
 .../io/DummyByteVersionedSerializer.java        |  55 ++
 .../cassandra/io/ShortVersionedSerializer.java  |  47 ++
 .../cassandra/io/sstable/SSTableLoader.java     |  31 +-
 .../locator/AbstractEndpointSnitch.java         |  19 +-
 .../locator/AbstractNetworkTopologySnitch.java  |   8 +-
 .../locator/AbstractReplicationStrategy.java    |  41 +-
 .../cassandra/locator/CloudstackSnitch.java     |  11 +-
 .../locator/DynamicEndpointSnitch.java          |  64 +-
 .../locator/DynamicEndpointSnitchMBean.java     |   2 +
 .../cassandra/locator/Ec2MultiRegionSnitch.java |  11 +
 .../org/apache/cassandra/locator/Ec2Snitch.java |  11 +-
 .../cassandra/locator/EndpointSnitchInfo.java   |   9 +-
 .../cassandra/locator/GoogleCloudSnitch.java    |  11 +-
 .../locator/GossipingPropertyFileSnitch.java    |  15 +-
 .../cassandra/locator/IEndpointSnitch.java      |  13 +-
 .../cassandra/locator/ILatencySubscriber.java   |   4 +-
 .../cassandra/locator/InetAddressAndPort.java   | 203 +++++
 .../apache/cassandra/locator/LocalStrategy.java |  11 +-
 .../locator/NetworkTopologyStrategy.java        |  23 +-
 .../locator/OldNetworkTopologyStrategy.java     |   5 +-
 .../cassandra/locator/PendingRangeMaps.java     |  57 +-
 .../cassandra/locator/PropertyFileSnitch.java   |  29 +-
 .../cassandra/locator/RackInferringSnitch.java  |  10 +-
 .../locator/ReconnectableSnitchHelper.java      |  60 +-
 .../apache/cassandra/locator/SeedProvider.java  |   3 +-
 .../cassandra/locator/SimpleSeedProvider.java   |   8 +-
 .../apache/cassandra/locator/SimpleSnitch.java  |   9 +-
 .../cassandra/locator/SimpleStrategy.java       |   7 +-
 .../apache/cassandra/locator/TokenMetadata.java | 221 +++--
 .../cassandra/metrics/ConnectionMetrics.java    |   8 +-
 .../cassandra/metrics/HintedHandoffMetrics.java |  25 +-
 .../cassandra/metrics/HintsServiceMetrics.java  |  11 +-
 .../cassandra/metrics/MessagingMetrics.java     |   4 +-
 .../cassandra/metrics/StreamingMetrics.java     |  11 +-
 .../apache/cassandra/net/BackPressureState.java |   4 +-
 .../cassandra/net/BackPressureStrategy.java     |   5 +-
 .../org/apache/cassandra/net/CallbackInfo.java  |   7 +-
 .../net/CompactEndpointSerializationHelper.java | 108 ++-
 .../cassandra/net/ForwardToContainer.java       |  43 +
 .../cassandra/net/ForwardToSerializer.java      |  86 ++
 .../apache/cassandra/net/IAsyncCallback.java    |   7 +-
 .../net/IAsyncCallbackWithFailure.java          |   5 +-
 .../org/apache/cassandra/net/IMessageSink.java  |   4 +-
 .../cassandra/net/MessageDeliveryTask.java      |  14 +-
 .../org/apache/cassandra/net/MessageIn.java     |  74 +-
 .../org/apache/cassandra/net/MessageOut.java    |  81 +-
 .../apache/cassandra/net/MessagingService.java  | 271 ++++--
 .../cassandra/net/MessagingServiceMBean.java    |  26 +
 .../org/apache/cassandra/net/ParameterType.java |  69 ++
 .../cassandra/net/RateBasedBackPressure.java    |   4 +-
 .../net/RateBasedBackPressureState.java         |   8 +-
 .../apache/cassandra/net/WriteCallbackInfo.java |   7 +-
 .../cassandra/net/async/HandshakeProtocol.java  |  18 +-
 .../net/async/InboundHandshakeHandler.java      |   7 +-
 .../cassandra/net/async/MessageInHandler.java   |  21 +-
 .../cassandra/net/async/MessageOutHandler.java  |  12 +-
 .../cassandra/net/async/NettyFactory.java       |  15 +-
 .../net/async/OutboundConnectionIdentifier.java |  48 +-
 .../net/async/OutboundMessagingConnection.java  |  13 +-
 .../net/async/OutboundMessagingPool.java        |  15 +-
 .../repair/AsymmetricLocalSyncTask.java         |   8 +-
 .../repair/AsymmetricRemoteSyncTask.java        |   6 +-
 .../cassandra/repair/AsymmetricSyncTask.java    |   8 +-
 .../apache/cassandra/repair/LocalSyncTask.java  |  13 +-
 .../org/apache/cassandra/repair/NodePair.java   |  25 +-
 .../apache/cassandra/repair/RemoteSyncTask.java |   4 +-
 .../org/apache/cassandra/repair/RepairJob.java  |  56 +-
 .../repair/RepairMessageVerbHandler.java        |   7 +-
 .../apache/cassandra/repair/RepairRunnable.java |  31 +-
 .../apache/cassandra/repair/RepairSession.java  |  38 +-
 .../apache/cassandra/repair/SnapshotTask.java   |  10 +-
 .../cassandra/repair/StreamingRepairTask.java   |  17 +-
 .../repair/SystemDistributedKeyspace.java       |  98 ++-
 .../apache/cassandra/repair/TreeResponse.java   |   7 +-
 .../apache/cassandra/repair/ValidationTask.java |   7 +-
 .../org/apache/cassandra/repair/Validator.java  |  12 +-
 .../repair/asymmetric/DifferenceHolder.java     |  16 +-
 .../repair/asymmetric/HostDifferences.java      |  14 +-
 .../asymmetric/IncomingRepairStreamTracker.java |   4 +-
 .../repair/asymmetric/PreferedNodeFilter.java   |   5 +-
 .../repair/asymmetric/ReduceHelper.java         |  42 +-
 .../repair/asymmetric/StreamFromOptions.java    |  16 +-
 .../repair/consistent/ConsistentSession.java    |  21 +-
 .../repair/consistent/CoordinatorSession.java   |  24 +-
 .../repair/consistent/CoordinatorSessions.java  |   4 +-
 .../repair/consistent/LocalSessionInfo.java     |   6 +-
 .../repair/consistent/LocalSessions.java        |  82 +-
 .../repair/consistent/SyncStatSummary.java      |  14 +-
 .../repair/messages/AsymmetricSyncRequest.java  |  25 +-
 .../repair/messages/FinalizePromise.java        |  18 +-
 .../messages/PrepareConsistentRequest.java      |  31 +-
 .../messages/PrepareConsistentResponse.java     |  17 +-
 .../cassandra/repair/messages/SyncComplete.java |   4 +-
 .../cassandra/repair/messages/SyncRequest.java  |  26 +-
 .../cassandra/schema/MigrationManager.java      |  24 +-
 .../apache/cassandra/schema/MigrationTask.java  |   6 +-
 .../cassandra/service/AbstractReadExecutor.java |  42 +-
 .../service/AbstractWriteResponseHandler.java   |  16 +-
 .../cassandra/service/ActiveRepairService.java  |  64 +-
 .../service/BatchlogResponseHandler.java        |   6 +-
 .../cassandra/service/CassandraDaemon.java      |   7 +-
 .../apache/cassandra/service/ClientState.java   |   2 +-
 .../apache/cassandra/service/DataResolver.java  |  18 +-
 .../DatacenterSyncWriteResponseHandler.java     |   8 +-
 .../service/DatacenterWriteResponseHandler.java |   8 +-
 .../service/IEndpointLifecycleSubscriber.java   |  12 +-
 .../cassandra/service/LoadBroadcaster.java      |  20 +-
 .../apache/cassandra/service/ReadCallback.java  |  20 +-
 .../apache/cassandra/service/StartupChecks.java |   4 +-
 .../apache/cassandra/service/StorageProxy.java  | 320 ++++----
 .../cassandra/service/StorageProxyMBean.java    |   3 +-
 .../cassandra/service/StorageService.java       | 818 ++++++++++++-------
 .../cassandra/service/StorageServiceMBean.java  |  57 +-
 .../apache/cassandra/service/TokenRange.java    |  50 +-
 .../cassandra/service/WriteResponseHandler.java |  12 +-
 .../service/paxos/PrepareCallback.java          |  10 +-
 .../cassandra/streaming/ProgressInfo.java       |  14 +-
 .../apache/cassandra/streaming/SessionInfo.java |  12 +-
 .../cassandra/streaming/SessionSummary.java     |  23 +-
 .../cassandra/streaming/StreamCoordinator.java  |  24 +-
 .../apache/cassandra/streaming/StreamEvent.java |   4 +-
 .../cassandra/streaming/StreamManager.java      |  10 +-
 .../apache/cassandra/streaming/StreamPlan.java  |  16 +-
 .../cassandra/streaming/StreamResultFuture.java |  15 +-
 .../cassandra/streaming/StreamSession.java      |  43 +-
 .../async/NettyStreamingMessageSender.java      |   4 +-
 .../async/StreamingInboundHandler.java          |  10 +-
 .../management/ProgressInfoCompositeData.java   |  30 +-
 .../SessionCompleteEventCompositeData.java      |   8 +-
 .../management/SessionInfoCompositeData.java    |  44 +-
 .../streaming/messages/FileMessageHeader.java   |  14 +-
 .../streaming/messages/OutgoingFileMessage.java |   2 +-
 .../streaming/messages/StreamInitMessage.java   |  12 +-
 .../tools/BulkLoadConnectionFactory.java        |  10 +-
 .../org/apache/cassandra/tools/BulkLoader.java  |  27 +-
 .../apache/cassandra/tools/LoaderOptions.java   |  77 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  63 +-
 .../org/apache/cassandra/tools/NodeTool.java    |  26 +
 .../tools/nodetool/DescribeCluster.java         |   4 +-
 .../cassandra/tools/nodetool/DescribeRing.java  |   4 +-
 .../tools/nodetool/FailureDetectorInfo.java     |   2 +-
 .../cassandra/tools/nodetool/GetEndpoints.java  |  18 +-
 .../cassandra/tools/nodetool/GossipInfo.java    |   4 +-
 .../tools/nodetool/HostStatWithPort.java        |  44 +
 .../cassandra/tools/nodetool/NetStats.java      |  10 +-
 .../cassandra/tools/nodetool/RemoveNode.java    |   6 +-
 .../cassandra/tools/nodetool/RepairAdmin.java   |   6 +-
 .../apache/cassandra/tools/nodetool/Ring.java   | 211 +++--
 .../tools/nodetool/SetHostStatWithPort.java     |  56 ++
 .../apache/cassandra/tools/nodetool/Status.java | 200 +++--
 .../cassandra/tracing/ExpiredTraceState.java    |   2 +-
 .../apache/cassandra/tracing/TraceKeyspace.java |  48 +-
 .../apache/cassandra/tracing/TraceState.java    |   6 +-
 .../cassandra/tracing/TraceStateImpl.java       |   4 +-
 .../org/apache/cassandra/tracing/Tracing.java   |  52 +-
 .../apache/cassandra/tracing/TracingImpl.java   |   3 +-
 .../org/apache/cassandra/transport/Event.java   |  21 +-
 .../cassandra/transport/ProtocolVersion.java    |   9 +-
 .../org/apache/cassandra/transport/Server.java  |  60 +-
 .../transport/messages/ErrorMessage.java        |  13 +-
 .../org/apache/cassandra/utils/FBUtilities.java |  53 +-
 .../apache/cassandra/utils/JMXServerUtils.java  |   2 +-
 .../org/apache/cassandra/utils/Mx4jTool.java    |   2 +-
 .../utils/NativeSSTableLoaderClient.java        |  46 +-
 .../org/apache/cassandra/utils/UUIDGen.java     |   2 +-
 test/conf/cassandra.yaml                        |   3 +-
 .../serialization/4.0/gms.EndpointState.bin     | Bin 73 -> 73 bytes
 test/data/serialization/4.0/gms.Gossip.bin      | Bin 158 -> 166 bytes
 .../serialization/4.0/service.SyncComplete.bin  | Bin 538 -> 554 bytes
 .../serialization/4.0/service.SyncRequest.bin   | Bin 227 -> 241 bytes
 .../4.0/service.ValidationComplete.bin          | Bin 1251 -> 1257 bytes
 .../4.0/service.ValidationRequest.bin           | Bin 167 -> 169 bytes
 .../locator/DynamicEndpointSnitchLongTest.java  |  15 +-
 .../cassandra/streaming/LongStreamingTest.java  |   4 +-
 .../test/microbench/PendingRangesBench.java     |  10 +-
 .../OffsetAwareConfigurationLoader.java         |  32 +
 test/unit/org/apache/cassandra/Util.java        |   9 +-
 .../batchlog/BatchlogEndpointFilterTest.java    |  84 +-
 .../cassandra/batchlog/BatchlogManagerTest.java |   6 +-
 .../config/DatabaseDescriptorRefTest.java       |   2 +
 .../org/apache/cassandra/cql3/CQLTester.java    |  12 +-
 .../cassandra/cql3/PreparedStatementsTest.java  |   1 -
 .../apache/cassandra/cql3/ViewComplexTest.java  |   8 +-
 .../cql3/validation/operations/CreateTest.java  |   8 +-
 .../org/apache/cassandra/db/CleanupTest.java    |  22 +-
 .../cassandra/db/DiskBoundaryManagerTest.java   |   6 +-
 .../apache/cassandra/db/ReadCommandTest.java    |  12 +-
 .../org/apache/cassandra/db/RowCacheTest.java   |   6 +-
 .../db/SystemKeyspaceMigrator40Test.java        | 192 +++++
 .../apache/cassandra/db/SystemKeyspaceTest.java |   6 +-
 .../compaction/AbstractPendingRepairTest.java   |   4 +-
 .../db/compaction/AntiCompactionTest.java       |   4 +-
 ...tionManagerGetSSTablesForValidationTest.java |   6 +-
 .../LeveledCompactionStrategyTest.java          |   4 +-
 .../apache/cassandra/db/view/ViewUtilsTest.java |  48 +-
 .../apache/cassandra/dht/BootStrapperTest.java  |  34 +-
 .../dht/RangeFetchMapCalculatorTest.java        | 116 +--
 .../cassandra/dht/StreamStateStoreTest.java     |   4 +-
 .../apache/cassandra/gms/ArrivalWindowTest.java |  15 +-
 .../apache/cassandra/gms/EndpointStateTest.java |   8 +-
 .../cassandra/gms/FailureDetectorTest.java      |   8 +-
 .../apache/cassandra/gms/GossipDigestTest.java  |   5 +-
 .../org/apache/cassandra/gms/GossiperTest.java  |   6 +-
 .../gms/PendingRangeCalculatorServiceTest.java  |   8 +-
 .../cassandra/gms/SerializationsTest.java       |  12 +-
 .../org/apache/cassandra/hints/HintTest.java    |  12 +-
 .../cassandra/hints/HintsServiceTest.java       |  14 +-
 .../io/sstable/CQLSSTableWriterTest.java        |   2 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |   2 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java |   2 +-
 .../cassandra/locator/CloudstackSnitchTest.java |   7 +-
 .../locator/DynamicEndpointSnitchTest.java      |  17 +-
 .../apache/cassandra/locator/EC2SnitchTest.java |   8 +-
 .../locator/GoogleCloudSnitchTest.java          |   7 +-
 .../GossipingPropertyFileSnitchTest.java        |  15 +-
 .../locator/InetAddressAndPortTest.java         | 143 ++++
 .../locator/NetworkTopologyStrategyTest.java    |  73 +-
 .../locator/OldNetworkTopologyStrategyTest.java |  21 +-
 .../cassandra/locator/PendingRangeMapsTest.java |  37 +-
 .../locator/PropertyFileSnitchTest.java         | 108 +--
 .../locator/ReconnectableSnitchHelperTest.java  |   2 +-
 .../ReplicationStrategyEndpointCacheTest.java   |  35 +-
 .../cassandra/locator/SimpleStrategyTest.java   |  19 +-
 .../cassandra/locator/TokenMetadataTest.java    |  49 +-
 .../metrics/HintedHandOffMetricsTest.java       |  10 +-
 .../CompactEndpointSerializationHelperTest.java |  72 ++
 .../cassandra/net/ForwardToContainerTest.java   | 100 +++
 test/unit/org/apache/cassandra/net/Matcher.java |   4 +-
 .../apache/cassandra/net/MatcherResponse.java   |  11 +-
 .../cassandra/net/MessagingServiceTest.java     |  82 +-
 .../cassandra/net/MockMessagingService.java     |  15 +-
 .../cassandra/net/MockMessagingServiceTest.java |   6 +-
 .../net/RateBasedBackPressureTest.java          |  42 +-
 .../cassandra/net/WriteCallbackInfoTest.java    |   4 +-
 .../cassandra/net/async/ChannelWriterTest.java  |   6 +-
 .../net/async/HandshakeHandlersTest.java        |   8 +-
 .../net/async/HandshakeProtocolTest.java        |   2 +-
 .../net/async/InboundHandshakeHandlerTest.java  |  18 +-
 .../net/async/MessageInHandlerTest.java         |  39 +-
 .../net/async/MessageOutHandlerTest.java        |  16 +-
 .../cassandra/net/async/NettyFactoryTest.java   |  15 +-
 .../net/async/OutboundHandshakeHandlerTest.java |   6 +-
 .../async/OutboundMessagingConnectionTest.java  |  48 +-
 .../net/async/OutboundMessagingPoolTest.java    |  11 +-
 .../cassandra/repair/AbstractRepairTest.java    |  20 +-
 .../cassandra/repair/LocalSyncTaskTest.java     |  12 +-
 .../cassandra/repair/RepairRunnableTest.java    |   3 +-
 .../cassandra/repair/RepairSessionTest.java     |   7 +-
 .../apache/cassandra/repair/ValidatorTest.java  |  12 +-
 .../repair/asymmetric/DifferenceHolderTest.java |  10 +-
 .../repair/asymmetric/ReduceHelperTest.java     |  74 +-
 .../asymmetric/StreamFromOptionsTest.java       |  36 +-
 .../AbstractConsistentSessionTest.java          |  20 +-
 .../consistent/CoordinatorSessionTest.java      |  30 +-
 .../consistent/CoordinatorSessionsTest.java     |  12 +-
 .../repair/consistent/LocalSessionAccessor.java |   4 +-
 .../repair/consistent/LocalSessionTest.java     |  17 +-
 .../consistent/PendingAntiCompactionTest.java   |   6 +-
 .../RepairMessageSerializationsTest.java        |  12 +-
 .../messages/RepairMessageSerializerTest.java   |  14 +-
 .../service/ActiveRepairServiceTest.java        |  58 +-
 .../cassandra/service/DataResolverTest.java     |  84 +-
 .../service/LeaveAndBootstrapTest.java          |  63 +-
 .../org/apache/cassandra/service/MoveTest.java  | 146 ++--
 .../service/ProtocolBetaVersionTest.java        |   2 +-
 .../cassandra/service/ReadExecutorTest.java     |   6 +-
 .../apache/cassandra/service/RemoveTest.java    |  14 +-
 .../cassandra/service/SerializationsTest.java   |  30 +-
 .../cassandra/service/StorageProxyTest.java     |   6 +-
 .../service/StorageServiceServerTest.java       | 170 ++--
 .../service/WriteResponseHandlerTest.java       |  25 +-
 .../cassandra/streaming/SessionInfoTest.java    |   4 +-
 .../streaming/StreamTransferTaskTest.java       |   6 +-
 .../streaming/StreamingTransferTest.java        |   4 +-
 .../async/NettyStreamingMessageSenderTest.java  |   8 +-
 .../async/StreamingInboundHandlerTest.java      |  12 +-
 .../apache/cassandra/tracing/TracingTest.java   |   3 +-
 .../cassandra/transport/ErrorMessageTest.java   |  20 +-
 .../transport/ProtocolVersionTest.java          |   9 +-
 .../cassandra/transport/SerDeserTest.java       |  12 +-
 .../apache/cassandra/utils/FBUtilitiesTest.java |   8 +-
 .../cassandra/stress/CompactionStress.java      |   4 +-
 .../cassandra/stress/settings/SettingsNode.java |  16 +-
 .../cassandra/stress/util/JavaDriverClient.java |   7 +-
 337 files changed, 6456 insertions(+), 3772 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b29ae78..d8ae88f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Allow storage port to be configurable per node (CASSANDRA-7544)
  * Make sub-range selection for non-frozen collections return null instead of empty (CASSANDRA-14182)
  * BloomFilter serialization format should not change byte ordering (CASSANDRA-9067)
  * Remove unused on-heap BloomFilter implementation (CASSANDRA-14152)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 06d73ea..c314030 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -39,7 +39,7 @@ New features
      This property can be modified at runtime through both JMX and the new `setconcurrentviewbuilders`
      and `getconcurrentviewbuilders` nodetool commands. See CASSANDRA-12245 for more details.
    - There is now a binary full query log based on Chronicle Queue that can be controlled using
-     nodetool enablefullquerylog, disablefullquerylog, and resetfullquerylog. The log 
+     nodetool enablefullquerylog, disablefullquerylog, and resetfullquerylog. The log
      contains all queries invoked, approximate time they were invoked, any parameters necessary
      to bind wildcard values, and all query options. A human readable version of the log can be
      dumped or tailed using the new bin/fqltool utility. The full query log is designed to be safe
@@ -103,6 +103,22 @@ Upgrading
 	  but blocks for up to a configurable number of milliseconds between disk flushes.
 	- nodetool clearsnapshot now required the --all flag to remove all snapshots.
 	  Previous behavior would delete all snapshots by default.
+    - Nodes are now identified by a combination of IP, and storage port.
+      Existing JMX APIs, nodetool, and system tables continue to work
+      and accept/return just an IP, but there is a new
+      version of each that works with the full unambiguous identifier.
+      You should prefer these over the deprecated ambiguous versions that only
+      work with an IP. This was done to support multiple instances per IP.
+      Additionally we are moving to only using a single port for encrypted and
+      unencrypted traffic and if you want multiple instances per IP you must
+      first switch encrypted traffic to the storage port and not a separate
+      encrypted port. If you want to use multiple instances per IP
+      with SSL you will need to use StartTLS on storage_port and set
+      outgoing_encrypted_port_source to gossip outbound connections
+      know what port to connect to for each instance. Before changing
+      storage port or native port at nodes you must first upgrade the entire cluster
+      and clients to 4.0 so they can handle the port not being consistent across
+      the cluster.
 
 Materialized Views
 -------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/bin/cqlsh.py
----------------------------------------------------------------------
diff --git a/bin/cqlsh.py b/bin/cqlsh.py
index 61ea160..3055110 100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@ -455,7 +455,8 @@ class Shell(cmd.Cmd):
                  single_statement=None,
                  request_timeout=DEFAULT_REQUEST_TIMEOUT_SECONDS,
                  protocol_version=None,
-                 connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS):
+                 connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS,
+                 allow_server_port_discovery=False):
         cmd.Cmd.__init__(self, completekey=completekey)
         self.hostname = hostname
         self.port = port
@@ -470,6 +471,7 @@ class Shell(cmd.Cmd):
         self.tracing_enabled = tracing_enabled
         self.page_size = self.default_page_size
         self.expand_enabled = expand_enabled
+        self.allow_server_port_discovery = allow_server_port_discovery
         if use_conn:
             self.conn = use_conn
         else:
@@ -482,6 +484,7 @@ class Shell(cmd.Cmd):
                                 load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
                                 control_connection_timeout=connect_timeout,
                                 connect_timeout=connect_timeout,
+                                allow_server_port_discovery=allow_server_port_discovery,
                                 **kwargs)
         self.owns_connection = not use_conn
 
@@ -1768,7 +1771,8 @@ class Shell(cmd.Cmd):
                          display_timezone=self.display_timezone,
                          max_trace_wait=self.max_trace_wait, ssl=self.ssl,
                          request_timeout=self.session.default_timeout,
-                         connect_timeout=self.conn.connect_timeout)
+                         connect_timeout=self.conn.connect_timeout,
+                         allow_server_port_discovery=self.allow_server_port_discovery)
         subshell.cmdloop()
         f.close()
 
@@ -2252,6 +2256,7 @@ def read_options(cmdlineargs, environment):
     optvalues.connect_timeout = option_with_default(configs.getint, 'connection', 'timeout', DEFAULT_CONNECT_TIMEOUT_SECONDS)
     optvalues.request_timeout = option_with_default(configs.getint, 'connection', 'request_timeout', DEFAULT_REQUEST_TIMEOUT_SECONDS)
     optvalues.execute = None
+    optvalues.allow_server_port_discovery = option_with_default(configs.getboolean, 'connection', 'allow_server_port_discovery', 'False')
 
     (options, arguments) = parser.parse_args(cmdlineargs, values=optvalues)
 
@@ -2415,7 +2420,8 @@ def main(options, hostname, port):
                       single_statement=options.execute,
                       request_timeout=options.request_timeout,
                       connect_timeout=options.connect_timeout,
-                      encoding=options.encoding)
+                      encoding=options.encoding,
+                      allow_server_port_discovery=options.allow_server_port_discovery)
     except KeyboardInterrupt:
         sys.exit('Connection aborted.')
     except CQL_ERRORS, e:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 8a033e0..5796868 100644
--- a/build.xml
+++ b/build.xml
@@ -306,7 +306,7 @@
       <!-- define the remote repositories we use -->
       <artifact:remoteRepository id="central"   url="${artifact.remoteRepository.central}"/>
       <artifact:remoteRepository id="apache"    url="${artifact.remoteRepository.apache}"/>
-      
+
       <macrodef name="install">
         <attribute name="pomFile"/>
         <attribute name="file"/>
@@ -437,7 +437,7 @@
           <dependency groupId="com.google.code.findbugs" artifactId="jsr305" version="2.0.2" />
           <dependency groupId="com.clearspring.analytics" artifactId="stream" version="2.5.2" />
 	  <!-- UPDATE AND UNCOMMENT ON THE DRIVER RELEASE, BEFORE 4.0 RELEASE
-          <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" version="3.0.1" classifier="shaded">
+          <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" version="4.0.0-SNAPSHOT" classifier="shaded">
             <exclusion groupId="io.netty" artifactId="netty-buffer"/>
             <exclusion groupId="io.netty" artifactId="netty-codec"/>
             <exclusion groupId="io.netty" artifactId="netty-handler"/>
@@ -522,7 +522,7 @@
       	<dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster"/>
       	<dependency groupId="com.google.code.findbugs" artifactId="jsr305"/>
         <dependency groupId="org.antlr" artifactId="antlr"/>
-	<!-- UPDATE AND UNCOMMENT ON THE DRIVER RELEASE, BEFORE 4.0 RELEASE	
+	<!-- UPDATE AND UNCOMMENT ON THE DRIVER RELEASE, BEFORE 4.0 RELEASE
         <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" classifier="shaded"/>
 	-->
         <dependency groupId="org.eclipse.jdt.core.compiler" artifactId="ecj"/>
@@ -543,7 +543,7 @@
         <dependency groupId="junit" artifactId="junit"/>
 	<!-- UPDATE AND UNCOMMENT ON THE DRIVER RELEASE, BEFORE 4.0 RELEASE
         <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" classifier="shaded"/>
-	-->     
+	-->
         <dependency groupId="io.netty" artifactId="netty-all"/>
         <dependency groupId="org.eclipse.jdt.core.compiler" artifactId="ecj"/>
         <dependency groupId="org.caffinitas.ohc" artifactId="ohc-core"/>
@@ -624,7 +624,7 @@
           <exclusion groupId="io.netty" artifactId="netty-transport"/>
         </dependency>
 	-->
-	
+
         <!-- don't need jna to run, but nice to have -->
         <dependency groupId="net.java.dev.jna" artifactId="jna"/>
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 80e4515..3bed3a6 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -416,7 +416,7 @@ seed_provider:
       parameters:
           # seeds is actually a comma-delimited list of addresses.
           # Ex: "<ip1>,<ip2>,<ip3>"
-          - seeds: "127.0.0.1"
+          - seeds: "127.0.0.1:7000"
 
 # For workloads with more data than can fit in memory, Cassandra's
 # bottleneck will be reads that need to fetch data from
@@ -945,6 +945,13 @@ dynamic_snitch_badness_threshold: 0.1
 # the keystore and truststore.  For instructions on generating these files, see:
 # http://download.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore
 #
+# If you are taking advantage of StartTLS outbound connections will have the issue that they can't know
+# what encrypted port to connect to in a foolproof way. outgoing_encrypted_port_source deals with this confusion
+# by allowing you to specify how you want a node to pick an outgoing port for intra-cluster connections.
+# Valid values are "gossip" and "yaml". Gossip will always connect to the storage port for a node that is
+# published via a gossip which is always going to be the plain storage port. "yaml" will always select
+# the port configured as ssl_storage_port on THIS node. If you want to use SSL and have different storage
+# ports across the cluster you must select "gossip" and use StartTLS on storage_port.
 server_encryption_options:
     # set to true for allowing secure incoming connections
     enabled: false

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/lib/cassandra-driver-core-3.3.2-0461ed35-SNAPSHOT-shaded.jar
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-core-3.3.2-0461ed35-SNAPSHOT-shaded.jar b/lib/cassandra-driver-core-3.3.2-0461ed35-SNAPSHOT-shaded.jar
deleted file mode 100644
index a7be9cb..0000000
Binary files a/lib/cassandra-driver-core-3.3.2-0461ed35-SNAPSHOT-shaded.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/lib/cassandra-driver-core-4.0.0-SNAPSHOT-shaded.jar
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-core-4.0.0-SNAPSHOT-shaded.jar b/lib/cassandra-driver-core-4.0.0-SNAPSHOT-shaded.jar
new file mode 100644
index 0000000..609c393
Binary files /dev/null and b/lib/cassandra-driver-core-4.0.0-SNAPSHOT-shaded.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/lib/cassandra-driver-internal-only-3.11.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.11.zip b/lib/cassandra-driver-internal-only-3.11.zip
deleted file mode 100644
index f7760af..0000000
Binary files a/lib/cassandra-driver-internal-only-3.11.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/lib/cassandra-driver-internal-only-3.12.0.post0-9ee88ded.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.12.0.post0-9ee88ded.zip b/lib/cassandra-driver-internal-only-3.12.0.post0-9ee88ded.zip
new file mode 100644
index 0000000..4aa91b7
Binary files /dev/null and b/lib/cassandra-driver-internal-only-3.12.0.post0-9ee88ded.zip differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index 807e970..f232bdc 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.batchlog;
 
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
@@ -34,6 +33,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.*;
@@ -250,7 +250,7 @@ public class BatchlogManager implements BatchlogManagerMBean
         int positionInPage = 0;
         ArrayList<ReplayingBatch> unfinishedBatches = new ArrayList<>(pageSize);
 
-        Set<InetAddress> hintedNodes = new HashSet<>();
+        Set<InetAddressAndPort> hintedNodes = new HashSet<>();
         Set<UUID> replayedBatches = new HashSet<>();
 
         // Sending out batches for replay without waiting for them, so that one stuck batch doesn't affect others
@@ -295,7 +295,7 @@ public class BatchlogManager implements BatchlogManagerMBean
         replayedBatches.forEach(BatchlogManager::remove);
     }
 
-    private void finishAndClearBatches(ArrayList<ReplayingBatch> batches, Set<InetAddress> hintedNodes, Set<UUID> replayedBatches)
+    private void finishAndClearBatches(ArrayList<ReplayingBatch> batches, Set<InetAddressAndPort> hintedNodes, Set<UUID> replayedBatches)
     {
         // schedule hints for timed out deliveries
         for (ReplayingBatch batch : batches)
@@ -330,7 +330,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             this.replayedBytes = addMutations(version, serializedMutations);
         }
 
-        public int replay(RateLimiter rateLimiter, Set<InetAddress> hintedNodes) throws IOException
+        public int replay(RateLimiter rateLimiter, Set<InetAddressAndPort> hintedNodes) throws IOException
         {
             logger.trace("Replaying batch {}", id);
 
@@ -348,7 +348,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             return replayHandlers.size();
         }
 
-        public void finish(Set<InetAddress> hintedNodes)
+        public void finish(Set<InetAddressAndPort> hintedNodes)
         {
             for (int i = 0; i < replayHandlers.size(); i++)
             {
@@ -396,7 +396,7 @@ public class BatchlogManager implements BatchlogManagerMBean
                 mutations.add(mutation);
         }
 
-        private void writeHintsForUndeliveredEndpoints(int startFrom, Set<InetAddress> hintedNodes)
+        private void writeHintsForUndeliveredEndpoints(int startFrom, Set<InetAddressAndPort> hintedNodes)
         {
             int gcgs = gcgs(mutations);
 
@@ -420,7 +420,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
         private static List<ReplayWriteResponseHandler<Mutation>> sendReplays(List<Mutation> mutations,
                                                                               long writtenAt,
-                                                                              Set<InetAddress> hintedNodes)
+                                                                              Set<InetAddressAndPort> hintedNodes)
         {
             List<ReplayWriteResponseHandler<Mutation>> handlers = new ArrayList<>(mutations.size());
             for (Mutation mutation : mutations)
@@ -440,15 +440,15 @@ public class BatchlogManager implements BatchlogManagerMBean
          */
         private static ReplayWriteResponseHandler<Mutation> sendSingleReplayMutation(final Mutation mutation,
                                                                                      long writtenAt,
-                                                                                     Set<InetAddress> hintedNodes)
+                                                                                     Set<InetAddressAndPort> hintedNodes)
         {
-            Set<InetAddress> liveEndpoints = new HashSet<>();
+            Set<InetAddressAndPort> liveEndpoints = new HashSet<>();
             String ks = mutation.getKeyspaceName();
             Token tk = mutation.key().getToken();
 
-            for (InetAddress endpoint : StorageService.instance.getNaturalAndPendingEndpoints(ks, tk))
+            for (InetAddressAndPort endpoint : StorageService.instance.getNaturalAndPendingEndpoints(ks, tk))
             {
-                if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+                if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
                 {
                     mutation.apply();
                 }
@@ -469,7 +469,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
             ReplayWriteResponseHandler<Mutation> handler = new ReplayWriteResponseHandler<>(liveEndpoints, System.nanoTime());
             MessageOut<Mutation> message = mutation.createMessage();
-            for (InetAddress endpoint : liveEndpoints)
+            for (InetAddressAndPort endpoint : liveEndpoints)
                 MessagingService.instance().sendRR(message, endpoint, handler, false);
             return handler;
         }
@@ -488,11 +488,11 @@ public class BatchlogManager implements BatchlogManagerMBean
          */
         private static class ReplayWriteResponseHandler<T> extends WriteResponseHandler<T>
         {
-            private final Set<InetAddress> undelivered = Collections.newSetFromMap(new ConcurrentHashMap<>());
+            private final Set<InetAddressAndPort> undelivered = Collections.newSetFromMap(new ConcurrentHashMap<>());
 
-            ReplayWriteResponseHandler(Collection<InetAddress> writeEndpoints, long queryStartNanoTime)
+            ReplayWriteResponseHandler(Collection<InetAddressAndPort> writeEndpoints, long queryStartNanoTime)
             {
-                super(writeEndpoints, Collections.<InetAddress>emptySet(), null, null, null, WriteType.UNLOGGED_BATCH, queryStartNanoTime);
+                super(writeEndpoints, Collections.<InetAddressAndPort>emptySet(), null, null, null, WriteType.UNLOGGED_BATCH, queryStartNanoTime);
                 undelivered.addAll(writeEndpoints);
             }
 
@@ -505,7 +505,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             @Override
             public void response(MessageIn<T> m)
             {
-                boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddress() : m.from);
+                boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddressAndPort() : m.from);
                 assert removed;
                 super.response(m);
             }
@@ -515,9 +515,9 @@ public class BatchlogManager implements BatchlogManagerMBean
     public static class EndpointFilter
     {
         private final String localRack;
-        private final Multimap<String, InetAddress> endpoints;
+        private final Multimap<String, InetAddressAndPort> endpoints;
 
-        public EndpointFilter(String localRack, Multimap<String, InetAddress> endpoints)
+        public EndpointFilter(String localRack, Multimap<String, InetAddressAndPort> endpoints)
         {
             this.localRack = localRack;
             this.endpoints = endpoints;
@@ -526,15 +526,15 @@ public class BatchlogManager implements BatchlogManagerMBean
         /**
          * @return list of candidates for batchlog hosting. If possible these will be two nodes from different racks.
          */
-        public Collection<InetAddress> filter()
+        public Collection<InetAddressAndPort> filter()
         {
             // special case for single-node data centers
             if (endpoints.values().size() == 1)
                 return endpoints.values();
 
             // strip out dead endpoints and localhost
-            ListMultimap<String, InetAddress> validated = ArrayListMultimap.create();
-            for (Map.Entry<String, InetAddress> entry : endpoints.entries())
+            ListMultimap<String, InetAddressAndPort> validated = ArrayListMultimap.create();
+            for (Map.Entry<String, InetAddressAndPort> entry : endpoints.entries())
                 if (isValid(entry.getValue()))
                     validated.put(entry.getKey(), entry.getValue());
 
@@ -554,7 +554,7 @@ public class BatchlogManager implements BatchlogManagerMBean
                  * pick two random nodes from there; we are guaranteed to have at least two nodes in the single remaining rack
                  * because of the preceding if block.
                  */
-                List<InetAddress> otherRack = Lists.newArrayList(validated.values());
+                List<InetAddressAndPort> otherRack = Lists.newArrayList(validated.values());
                 shuffle(otherRack);
                 return otherRack.subList(0, 2);
             }
@@ -572,10 +572,10 @@ public class BatchlogManager implements BatchlogManagerMBean
             }
 
             // grab a random member of up to two racks
-            List<InetAddress> result = new ArrayList<>(2);
+            List<InetAddressAndPort> result = new ArrayList<>(2);
             for (String rack : Iterables.limit(racks, 2))
             {
-                List<InetAddress> rackMembers = validated.get(rack);
+                List<InetAddressAndPort> rackMembers = validated.get(rack);
                 result.add(rackMembers.get(getRandomInt(rackMembers.size())));
             }
 
@@ -583,9 +583,9 @@ public class BatchlogManager implements BatchlogManagerMBean
         }
 
         @VisibleForTesting
-        protected boolean isValid(InetAddress input)
+        protected boolean isValid(InetAddressAndPort input)
         {
-            return !input.equals(FBUtilities.getBroadcastAddress()) && FailureDetector.instance.isAlive(input);
+            return !input.equals(FBUtilities.getBroadcastAddressAndPort()) && FailureDetector.instance.isAlive(input);
         }
 
         @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/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 a656d1f..9012e3a 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -55,6 +55,7 @@ import org.apache.cassandra.io.util.SsdDiskOptimizationStrategy;
 import org.apache.cassandra.locator.DynamicEndpointSnitch;
 import org.apache.cassandra.locator.EndpointSnitchInfo;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.SeedProvider;
 import org.apache.cassandra.net.BackPressureStrategy;
 import org.apache.cassandra.net.RateBasedBackPressure;
@@ -110,7 +111,7 @@ public class DatabaseDescriptor
     private static long indexSummaryCapacityInMB;
 
     private static String localDC;
-    private static Comparator<InetAddress> localComparator;
+    private static Comparator<InetAddressAndPort> localComparator;
     private static EncryptionContext encryptionContext;
     private static boolean hasLoggedConfig;
 
@@ -307,6 +308,7 @@ public class DatabaseDescriptor
 
     private static void applyAll() throws ConfigurationException
     {
+        //InetAddressAndPort cares that applySimpleConfig runs first
         applySimpleConfig();
 
         applyPartitioner();
@@ -324,6 +326,9 @@ public class DatabaseDescriptor
 
     private static void applySimpleConfig()
     {
+        //Doing this first before all other things in case other pieces of config want to construct
+        //InetAddressAndPort and get the right defaults
+        InetAddressAndPort.initializeDefaultPort(getStoragePort());
 
         if (conf.commitlog_sync == null)
         {
@@ -827,7 +832,7 @@ public class DatabaseDescriptor
         }
         else
         {
-            rpcAddress = FBUtilities.getLocalAddress();
+            rpcAddress = FBUtilities.getJustLocalAddress();
         }
 
         /* RPC address to broadcast */
@@ -956,10 +961,10 @@ public class DatabaseDescriptor
         snitch = createEndpointSnitch(conf.dynamic_snitch, conf.endpoint_snitch);
         EndpointSnitchInfo.create();
 
-        localDC = snitch.getDatacenter(FBUtilities.getBroadcastAddress());
-        localComparator = new Comparator<InetAddress>()
+        localDC = snitch.getDatacenter(FBUtilities.getBroadcastAddressAndPort());
+        localComparator = new Comparator<InetAddressAndPort>()
         {
-            public int compare(InetAddress endpoint1, InetAddress endpoint2)
+            public int compare(InetAddressAndPort endpoint1, InetAddressAndPort endpoint2)
             {
                 boolean local1 = localDC.equals(snitch.getDatacenter(endpoint1));
                 boolean local2 = localDC.equals(snitch.getDatacenter(endpoint2));
@@ -1319,14 +1324,14 @@ public class DatabaseDescriptor
         return conf.num_tokens;
     }
 
-    public static InetAddress getReplaceAddress()
+    public static InetAddressAndPort getReplaceAddress()
     {
         try
         {
             if (System.getProperty(Config.PROPERTY_PREFIX + "replace_address", null) != null)
-                return InetAddress.getByName(System.getProperty(Config.PROPERTY_PREFIX + "replace_address", null));
+                return InetAddressAndPort.getByName(System.getProperty(Config.PROPERTY_PREFIX + "replace_address", null));
             else if (System.getProperty(Config.PROPERTY_PREFIX + "replace_address_first_boot", null) != null)
-                return InetAddress.getByName(System.getProperty(Config.PROPERTY_PREFIX + "replace_address_first_boot", null));
+                return InetAddressAndPort.getByName(System.getProperty(Config.PROPERTY_PREFIX + "replace_address_first_boot", null));
             return null;
         }
         catch (UnknownHostException e)
@@ -1651,9 +1656,9 @@ public class DatabaseDescriptor
         return conf.saved_caches_directory;
     }
 
-    public static Set<InetAddress> getSeeds()
+    public static Set<InetAddressAndPort> getSeeds()
     {
-        return ImmutableSet.<InetAddress>builder().addAll(seedProvider.getSeeds()).build();
+        return ImmutableSet.<InetAddressAndPort>builder().addAll(seedProvider.getSeeds()).build();
     }
 
     public static InetAddress getListenAddress()
@@ -2206,7 +2211,7 @@ public class DatabaseDescriptor
         return localDC;
     }
 
-    public static Comparator<InetAddress> getLocalComparator()
+    public static Comparator<InetAddressAndPort> getLocalComparator()
     {
         return localComparator;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/ConsistencyLevel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
index 2214d8d..f93e737 100644
--- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
+++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -28,6 +27,7 @@ import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.ReadRepairDecision;
@@ -145,21 +145,21 @@ public enum ConsistencyLevel
         return isDCLocal;
     }
 
-    public boolean isLocal(InetAddress endpoint)
+    public boolean isLocal(InetAddressAndPort endpoint)
     {
         return DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint));
     }
 
-    public int countLocalEndpoints(Iterable<InetAddress> liveEndpoints)
+    public int countLocalEndpoints(Iterable<InetAddressAndPort> liveEndpoints)
     {
         int count = 0;
-        for (InetAddress endpoint : liveEndpoints)
+        for (InetAddressAndPort endpoint : liveEndpoints)
             if (isLocal(endpoint))
                 count++;
         return count;
     }
 
-    private Map<String, Integer> countPerDCEndpoints(Keyspace keyspace, Iterable<InetAddress> liveEndpoints)
+    private Map<String, Integer> countPerDCEndpoints(Keyspace keyspace, Iterable<InetAddressAndPort> liveEndpoints)
     {
         NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
 
@@ -167,7 +167,7 @@ public enum ConsistencyLevel
         for (String dc: strategy.getDatacenters())
             dcEndpoints.put(dc, 0);
 
-        for (InetAddress endpoint : liveEndpoints)
+        for (InetAddressAndPort endpoint : liveEndpoints)
         {
             String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint);
             dcEndpoints.put(dc, dcEndpoints.get(dc) + 1);
@@ -175,12 +175,12 @@ public enum ConsistencyLevel
         return dcEndpoints;
     }
 
-    public List<InetAddress> filterForQuery(Keyspace keyspace, List<InetAddress> liveEndpoints)
+    public List<InetAddressAndPort> filterForQuery(Keyspace keyspace, List<InetAddressAndPort> liveEndpoints)
     {
         return filterForQuery(keyspace, liveEndpoints, ReadRepairDecision.NONE);
     }
 
-    public List<InetAddress> filterForQuery(Keyspace keyspace, List<InetAddress> liveEndpoints, ReadRepairDecision readRepair)
+    public List<InetAddressAndPort> filterForQuery(Keyspace keyspace, List<InetAddressAndPort> liveEndpoints, ReadRepairDecision readRepair)
     {
         /*
          * If we are doing an each quorum query, we have to make sure that the endpoints we select
@@ -206,9 +206,9 @@ public enum ConsistencyLevel
             case GLOBAL:
                 return liveEndpoints;
             case DC_LOCAL:
-                List<InetAddress> local = new ArrayList<InetAddress>();
-                List<InetAddress> other = new ArrayList<InetAddress>();
-                for (InetAddress add : liveEndpoints)
+                List<InetAddressAndPort> local = new ArrayList<>();
+                List<InetAddressAndPort> other = new ArrayList<>();
+                for (InetAddressAndPort add : liveEndpoints)
                 {
                     if (isLocal(add))
                         local.add(add);
@@ -225,7 +225,7 @@ public enum ConsistencyLevel
         }
     }
 
-    private List<InetAddress> filterForEachQuorum(Keyspace keyspace, List<InetAddress> liveEndpoints, ReadRepairDecision readRepair)
+    private List<InetAddressAndPort> filterForEachQuorum(Keyspace keyspace, List<InetAddressAndPort> liveEndpoints, ReadRepairDecision readRepair)
     {
         NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
 
@@ -233,20 +233,20 @@ public enum ConsistencyLevel
         if (readRepair == ReadRepairDecision.GLOBAL)
             return liveEndpoints;
 
-        Map<String, List<InetAddress>> dcsEndpoints = new HashMap<>();
+        Map<String, List<InetAddressAndPort>> dcsEndpoints = new HashMap<>();
         for (String dc: strategy.getDatacenters())
             dcsEndpoints.put(dc, new ArrayList<>());
 
-        for (InetAddress add : liveEndpoints)
+        for (InetAddressAndPort add : liveEndpoints)
         {
             String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(add);
             dcsEndpoints.get(dc).add(add);
         }
 
-        List<InetAddress> waitSet = new ArrayList<>();
-        for (Map.Entry<String, List<InetAddress>> dcEndpoints : dcsEndpoints.entrySet())
+        List<InetAddressAndPort> waitSet = new ArrayList<>();
+        for (Map.Entry<String, List<InetAddressAndPort>> dcEndpoints : dcsEndpoints.entrySet())
         {
-            List<InetAddress> dcEndpoint = dcEndpoints.getValue();
+            List<InetAddressAndPort> dcEndpoint = dcEndpoints.getValue();
             if (readRepair == ReadRepairDecision.DC_LOCAL && dcEndpoints.getKey().equals(DatabaseDescriptor.getLocalDataCenter()))
                 waitSet.addAll(dcEndpoint);
             else
@@ -256,7 +256,7 @@ public enum ConsistencyLevel
         return waitSet;
     }
 
-    public boolean isSufficientLiveNodes(Keyspace keyspace, Iterable<InetAddress> liveEndpoints)
+    public boolean isSufficientLiveNodes(Keyspace keyspace, Iterable<InetAddressAndPort> liveEndpoints)
     {
         switch (this)
         {
@@ -283,7 +283,7 @@ public enum ConsistencyLevel
         }
     }
 
-    public void assureSufficientLiveNodes(Keyspace keyspace, Iterable<InetAddress> liveEndpoints) throws UnavailableException
+    public void assureSufficientLiveNodes(Keyspace keyspace, Iterable<InetAddressAndPort> liveEndpoints) throws UnavailableException
     {
         int blockFor = blockFor(keyspace);
         switch (this)
@@ -302,7 +302,7 @@ public enum ConsistencyLevel
                     if (logger.isTraceEnabled())
                     {
                         StringBuilder builder = new StringBuilder("Local replicas [");
-                        for (InetAddress endpoint : liveEndpoints)
+                        for (InetAddressAndPort endpoint : liveEndpoints)
                         {
                             if (isLocal(endpoint))
                                 builder.append(endpoint).append(",");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
index bd273e4..95d7916 100644
--- a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
@@ -37,7 +37,7 @@ public class CounterMutationVerbHandler implements IVerbHandler<CounterMutation>
         final CounterMutation cm = message.payload;
         logger.trace("Applying forwarded {}", cm);
 
-        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
         // We should not wait for the result of the write in this thread,
         // otherwise we could have a distributed deadlock between replicas
         // running this VerbHandler (see #4578).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
index 03cbf7b..72b5e2a 100644
--- a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
+++ b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
@@ -80,14 +80,14 @@ public class DiskBoundaryManager
                 && !StorageService.isReplacingSameAddress()) // When replacing same address, the node marks itself as UN locally
             {
                 PendingRangeCalculatorService.instance.blockUntilFinished();
-                localRanges = tmd.getPendingRanges(cfs.keyspace.getName(), FBUtilities.getBroadcastAddress());
+                localRanges = tmd.getPendingRanges(cfs.keyspace.getName(), FBUtilities.getBroadcastAddressAndPort());
             }
             else
             {
                 // Reason we use use the future settled TMD is that if we decommission a node, we want to stream
                 // from that node to the correct location on disk, if we didn't, we would put new files in the wrong places.
                 // We do this to minimize the amount of data we need to move in rebalancedisks once everything settled
-                localRanges = cfs.keyspace.getReplicationStrategy().getAddressRanges(tmd.cloneAfterAllSettled()).get(FBUtilities.getBroadcastAddress());
+                localRanges = cfs.keyspace.getReplicationStrategy().getAddressRanges(tmd.cloneAfterAllSettled()).get(FBUtilities.getBroadcastAddressAndPort());
             }
             logger.debug("Got local ranges {} (ringVersion = {})", localRanges, ringVersion);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index cebf6eb..ae778f1 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -352,6 +352,7 @@ public class Keyspace
 
     private void createReplicationStrategy(KeyspaceMetadata ksm)
     {
+        logger.info("Creating replication strategy " + ksm .name + " params " + ksm.params);
         replicationStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,
                                                                                     ksm.params.replication.klass,
                                                                                     StorageService.instance.getTokenMetadata(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/Mutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java
index a6a920c..6195fe4 100644
--- a/src/java/org/apache/cassandra/db/Mutation.java
+++ b/src/java/org/apache/cassandra/db/Mutation.java
@@ -43,9 +43,6 @@ public class Mutation implements IMutation
 {
     public static final MutationSerializer serializer = new MutationSerializer();
 
-    public static final String FORWARD_TO = "FWD_TO";
-    public static final String FORWARD_FROM = "FWD_FRM";
-
     // todo this is redundant
     // when we remove it, also restore SerializationsTest.testMutationRead to not regenerate new Mutations each test
     private final String keyspaceName;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/MutationVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MutationVerbHandler.java b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
index 59247a2..8386048 100644
--- a/src/java/org/apache/cassandra/db/MutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
@@ -17,18 +17,17 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInputStream;
 import java.io.IOException;
-import java.net.InetAddress;
+import java.util.Iterator;
 
 import org.apache.cassandra.exceptions.WriteTimeoutException;
-import org.apache.cassandra.io.util.FastByteArrayInputStream;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.*;
 import org.apache.cassandra.tracing.Tracing;
 
 public class MutationVerbHandler implements IVerbHandler<Mutation>
 {
-    private void reply(int id, InetAddress replyTo)
+    private void reply(int id, InetAddressAndPort replyTo)
     {
         Tracing.trace("Enqueuing response to {}", replyTo);
         MessagingService.instance().sendReply(WriteResponse.createMessage(), id, replyTo);
@@ -42,18 +41,19 @@ public class MutationVerbHandler implements IVerbHandler<Mutation>
     public void doVerb(MessageIn<Mutation> message, int id)  throws IOException
     {
         // Check if there were any forwarding headers in this message
-        byte[] from = message.parameters.get(Mutation.FORWARD_FROM);
-        InetAddress replyTo;
+        InetAddressAndPort from = (InetAddressAndPort)message.parameters.get(ParameterType.FORWARD_FROM);
+        InetAddressAndPort replyTo;
         if (from == null)
         {
             replyTo = message.from;
-            byte[] forwardBytes = message.parameters.get(Mutation.FORWARD_TO);
-            if (forwardBytes != null)
-                forwardToLocalNodes(message.payload, message.verb, forwardBytes, message.from);
+            ForwardToContainer forwardTo = (ForwardToContainer)message.parameters.get(ParameterType.FORWARD_TO);
+            if (forwardTo != null)
+                forwardToLocalNodes(message.payload, message.verb, forwardTo, message.from);
         }
         else
         {
-            replyTo = InetAddress.getByAddress(from);
+
+            replyTo = from;
         }
 
         try
@@ -69,22 +69,17 @@ public class MutationVerbHandler implements IVerbHandler<Mutation>
         }
     }
 
-    private static void forwardToLocalNodes(Mutation mutation, MessagingService.Verb verb, byte[] forwardBytes, InetAddress from) throws IOException
+    private static void forwardToLocalNodes(Mutation mutation, MessagingService.Verb verb, ForwardToContainer forwardTo, InetAddressAndPort from) throws IOException
     {
-        try (DataInputStream in = new DataInputStream(new FastByteArrayInputStream(forwardBytes)))
+        // tell the recipients who to send their ack to
+        MessageOut<Mutation> message = new MessageOut<>(verb, mutation, Mutation.serializer).withParameter(ParameterType.FORWARD_FROM, from);
+        Iterator<InetAddressAndPort> iterator = forwardTo.targets.iterator();
+        // Send a message to each of the addresses on our Forward List
+        for (int i = 0; i < forwardTo.targets.size(); i++)
         {
-            int size = in.readInt();
-
-            // tell the recipients who to send their ack to
-            MessageOut<Mutation> message = new MessageOut<>(verb, mutation, Mutation.serializer).withParameter(Mutation.FORWARD_FROM, from.getAddress());
-            // Send a message to each of the addresses on our Forward List
-            for (int i = 0; i < size; i++)
-            {
-                InetAddress address = CompactEndpointSerializationHelper.deserialize(in);
-                int id = in.readInt();
-                Tracing.trace("Enqueuing forwarded write to {}", address);
-                MessagingService.instance().sendOneWay(message, id, address);
-            }
+            InetAddressAndPort address = iterator.next();
+            Tracing.trace("Enqueuing forwarded write to {}", address);
+            MessagingService.instance().sendOneWay(message, forwardTo.messageIds[i], address);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
index 066b2fe..151e7d3 100644
--- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
+++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
@@ -60,7 +60,7 @@ public class SizeEstimatesRecorder extends SchemaChangeListener implements Runna
     public void run()
     {
         TokenMetadata metadata = StorageService.instance.getTokenMetadata().cloneOnlyTokenMap();
-        if (!metadata.isMember(FBUtilities.getBroadcastAddress()))
+        if (!metadata.isMember(FBUtilities.getBroadcastAddressAndPort()))
         {
             logger.debug("Node is not part of the ring; not recording size estimates");
             return;
@@ -71,7 +71,7 @@ public class SizeEstimatesRecorder extends SchemaChangeListener implements Runna
         for (Keyspace keyspace : Keyspace.nonLocalStrategy())
         {
             Collection<Range<Token>> localRanges = StorageService.instance.getPrimaryRangesForEndpoint(keyspace.getName(),
-                    FBUtilities.getBroadcastAddress());
+                    FBUtilities.getBroadcastAddressAndPort());
             for (ColumnFamilyStore table : keyspace.getColumnFamilyStores())
             {
                 long start = System.nanoTime();


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


[17/19] cassandra git commit: Allow storage port to be configurable per node

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/exceptions/ReadFailureException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/ReadFailureException.java b/src/java/org/apache/cassandra/exceptions/ReadFailureException.java
index 82885e3..72242fd 100644
--- a/src/java/org/apache/cassandra/exceptions/ReadFailureException.java
+++ b/src/java/org/apache/cassandra/exceptions/ReadFailureException.java
@@ -17,16 +17,16 @@
  */
 package org.apache.cassandra.exceptions;
 
-import java.net.InetAddress;
 import java.util.Map;
 
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public class ReadFailureException extends RequestFailureException
 {
     public final boolean dataPresent;
 
-    public ReadFailureException(ConsistencyLevel consistency, int received, int blockFor, boolean dataPresent, Map<InetAddress, RequestFailureReason> failureReasonByEndpoint)
+    public ReadFailureException(ConsistencyLevel consistency, int received, int blockFor, boolean dataPresent, Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint)
     {
         super(ExceptionCode.READ_FAILURE, consistency, received, blockFor, failureReasonByEndpoint);
         this.dataPresent = dataPresent;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureException.java b/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
index 1a5289c..2b57a75 100644
--- a/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
+++ b/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
@@ -17,20 +17,20 @@
  */
 package org.apache.cassandra.exceptions;
 
-import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public class RequestFailureException extends RequestExecutionException
 {
     public final ConsistencyLevel consistency;
     public final int received;
     public final int blockFor;
-    public final Map<InetAddress, RequestFailureReason> failureReasonByEndpoint;
+    public final Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint;
 
-    protected RequestFailureException(ExceptionCode code, ConsistencyLevel consistency, int received, int blockFor, Map<InetAddress, RequestFailureReason> failureReasonByEndpoint)
+    protected RequestFailureException(ExceptionCode code, ConsistencyLevel consistency, int received, int blockFor, Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint)
     {
         super(code, String.format("Operation failed - received %d responses and %d failures", received, failureReasonByEndpoint.size()));
         this.consistency = consistency;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/exceptions/WriteFailureException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/WriteFailureException.java b/src/java/org/apache/cassandra/exceptions/WriteFailureException.java
index 1a857fe..a7dc66a 100644
--- a/src/java/org/apache/cassandra/exceptions/WriteFailureException.java
+++ b/src/java/org/apache/cassandra/exceptions/WriteFailureException.java
@@ -17,17 +17,17 @@
  */
 package org.apache.cassandra.exceptions;
 
-import java.net.InetAddress;
 import java.util.Map;
 
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public class WriteFailureException extends RequestFailureException
 {
     public final WriteType writeType;
 
-    public WriteFailureException(ConsistencyLevel consistency, int received, int blockFor, WriteType writeType, Map<InetAddress, RequestFailureReason> failureReasonByEndpoint)
+    public WriteFailureException(ConsistencyLevel consistency, int received, int blockFor, WriteType writeType, Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint)
     {
         super(ExceptionCode.WRITE_FAILURE, consistency, received, blockFor, failureReasonByEndpoint);
         this.writeType = writeType;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/ApplicationState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/ApplicationState.java b/src/java/org/apache/cassandra/gms/ApplicationState.java
index ade9208..211387d 100644
--- a/src/java/org/apache/cassandra/gms/ApplicationState.java
+++ b/src/java/org/apache/cassandra/gms/ApplicationState.java
@@ -19,15 +19,15 @@ package org.apache.cassandra.gms;
 
 public enum ApplicationState
 {
-    STATUS,
+    @Deprecated STATUS, //Deprecated and unsued in 4.0, stop publishing in 5.0, reclaim in 6.0
     LOAD,
     SCHEMA,
     DC,
     RACK,
     RELEASE_VERSION,
     REMOVAL_COORDINATOR,
-    INTERNAL_IP,
-    RPC_ADDRESS,
+    @Deprecated INTERNAL_IP, //Deprecated and unused in 4.0, stop publishing in 5.0, reclaim in 6.0
+    @Deprecated RPC_ADDRESS, // ^ Same
     X_11_PADDING, // padding specifically for 1.1
     SEVERITY,
     NET_VERSION,
@@ -35,8 +35,9 @@ public enum ApplicationState
     TOKENS,
     RPC_READY,
     // pad to allow adding new states to existing cluster
-    X1,
-    X2,
+    INTERNAL_ADDRESS_AND_PORT, //Replacement for INTERNAL_IP with up to two ports
+    NATIVE_ADDRESS_AND_PORT, //Replacement for RPC_ADDRESS
+    STATUS_WITH_PORT, //Replacement for STATUS
     X3,
     X4,
     X5,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/EndpointState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/EndpointState.java b/src/java/org/apache/cassandra/gms/EndpointState.java
index 847041f..1085447 100644
--- a/src/java/org/apache/cassandra/gms/EndpointState.java
+++ b/src/java/org/apache/cassandra/gms/EndpointState.java
@@ -146,9 +146,15 @@ public class EndpointState
 
     public String getStatus()
     {
-        VersionedValue status = getApplicationState(ApplicationState.STATUS);
+        VersionedValue status = getApplicationState(ApplicationState.STATUS_WITH_PORT);
         if (status == null)
+        {
+            status = getApplicationState(ApplicationState.STATUS);
+        }
+        if (status == null)
+        {
             return "";
+        }
 
         String[] pieces = status.value.split(VersionedValue.DELIMITER_STR, -1);
         assert (pieces.length > 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/FailureDetector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java
index b895082..e567b7b 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetector.java
@@ -22,7 +22,6 @@ import java.nio.file.StandardOpenOption;
 import java.nio.file.Path;
 import java.io.*;
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
@@ -38,7 +37,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.Clock;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -79,7 +78,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
     // change.
     private final double PHI_FACTOR = 1.0 / Math.log(10.0); // 0.434...
 
-    private final ConcurrentHashMap<InetAddress, ArrivalWindow> arrivalSamples = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<InetAddressAndPort, ArrivalWindow> arrivalSamples = new ConcurrentHashMap<>();
     private final List<IFailureDetectionEventListener> fdEvntListeners = new CopyOnWriteArrayList<>();
 
     public FailureDetector()
@@ -112,10 +111,20 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
 
     public String getAllEndpointStates()
     {
+        return getAllEndpointStates(false);
+    }
+
+    public String getAllEndpointStatesWithPort()
+    {
+        return getAllEndpointStates(true);
+    }
+
+    public String getAllEndpointStates(boolean withPort)
+    {
         StringBuilder sb = new StringBuilder();
-        for (Map.Entry<InetAddress, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet())
         {
-            sb.append(entry.getKey()).append("\n");
+            sb.append(entry.getKey().toString(withPort)).append("\n");
             appendEndpointState(sb, entry.getValue());
         }
         return sb.toString();
@@ -123,13 +132,23 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
 
     public Map<String, String> getSimpleStates()
     {
+        return getSimpleStates(false);
+    }
+
+    public Map<String, String> getSimpleStatesWithPort()
+    {
+        return getSimpleStates(true);
+    }
+
+    private Map<String, String> getSimpleStates(boolean withPort)
+    {
         Map<String, String> nodesStatus = new HashMap<String, String>(Gossiper.instance.endpointStateMap.size());
-        for (Map.Entry<InetAddress, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet())
         {
             if (entry.getValue().isAlive())
-                nodesStatus.put(entry.getKey().toString(), "UP");
+                nodesStatus.put(entry.getKey().toString(withPort), "UP");
             else
-                nodesStatus.put(entry.getKey().toString(), "DOWN");
+                nodesStatus.put(entry.getKey().toString(withPort), "DOWN");
         }
         return nodesStatus;
     }
@@ -137,7 +156,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
     public int getDownEndpointCount()
     {
         int count = 0;
-        for (Map.Entry<InetAddress, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet())
         {
             if (!entry.getValue().isAlive())
                 count++;
@@ -148,7 +167,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
     public int getUpEndpointCount()
     {
         int count = 0;
-        for (Map.Entry<InetAddress, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet())
         {
             if (entry.getValue().isAlive())
                 count++;
@@ -159,13 +178,24 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
     @Override
     public TabularData getPhiValues() throws OpenDataException
     {
+        return getPhiValues(false);
+    }
+
+    @Override
+    public TabularData getPhiValuesWithPort() throws OpenDataException
+    {
+        return getPhiValues(true);
+    }
+
+    private TabularData getPhiValues(boolean withPort) throws OpenDataException
+    {
         final CompositeType ct = new CompositeType("Node", "Node",
                 new String[]{"Endpoint", "PHI"},
                 new String[]{"IP of the endpoint", "PHI value"},
                 new OpenType[]{SimpleType.STRING, SimpleType.DOUBLE});
         final TabularDataSupport results = new TabularDataSupport(new TabularType("PhiList", "PhiList", ct, new String[]{"Endpoint"}));
 
-        for (final Map.Entry<InetAddress, ArrivalWindow> entry : arrivalSamples.entrySet())
+        for (final Map.Entry<InetAddressAndPort, ArrivalWindow> entry : arrivalSamples.entrySet())
         {
             final ArrivalWindow window = entry.getValue();
             if (window.mean() > 0)
@@ -176,7 +206,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
                     // returned values are scaled by PHI_FACTOR so that the are on the same scale as PhiConvictThreshold
                     final CompositeData data = new CompositeDataSupport(ct,
                             new String[]{"Endpoint", "PHI"},
-                            new Object[]{entry.getKey().toString(), phi * PHI_FACTOR});
+                            new Object[]{entry.getKey().toString(withPort), phi * PHI_FACTOR});
                     results.put(data);
                 }
             }
@@ -187,7 +217,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
     public String getEndpointState(String address) throws UnknownHostException
     {
         StringBuilder sb = new StringBuilder();
-        EndpointState endpointState = Gossiper.instance.getEndpointStateForEndpoint(InetAddress.getByName(address));
+        EndpointState endpointState = Gossiper.instance.getEndpointStateForEndpoint(InetAddressAndPort.getByName(address));
         appendEndpointState(sb, endpointState);
         return sb.toString();
     }
@@ -243,9 +273,9 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
         return DatabaseDescriptor.getPhiConvictThreshold();
     }
 
-    public boolean isAlive(InetAddress ep)
+    public boolean isAlive(InetAddressAndPort ep)
     {
-        if (ep.equals(FBUtilities.getBroadcastAddress()))
+        if (ep.equals(FBUtilities.getBroadcastAddressAndPort()))
             return true;
 
         EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(ep);
@@ -257,7 +287,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
         return epState != null && epState.isAlive();
     }
 
-    public void report(InetAddress ep)
+    public void report(InetAddressAndPort ep)
     {
         long now = Clock.instance.nanoTime();
         ArrivalWindow heartbeatWindow = arrivalSamples.get(ep);
@@ -279,7 +309,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
             logger.trace("Average for {} is {}ns", ep, heartbeatWindow.mean());
     }
 
-    public void interpret(InetAddress ep)
+    public void interpret(InetAddressAndPort ep)
     {
         ArrivalWindow hbWnd = arrivalSamples.get(ep);
         if (hbWnd == null)
@@ -324,7 +354,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
         }
     }
 
-    public void forceConviction(InetAddress ep)
+    public void forceConviction(InetAddressAndPort ep)
     {
         logger.debug("Forcing conviction of {}", ep);
         for (IFailureDetectionEventListener listener : fdEvntListeners)
@@ -333,7 +363,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
         }
     }
 
-    public void remove(InetAddress ep)
+    public void remove(InetAddressAndPort ep)
     {
         arrivalSamples.remove(ep);
     }
@@ -351,10 +381,10 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
     public String toString()
     {
         StringBuilder sb = new StringBuilder();
-        Set<InetAddress> eps = arrivalSamples.keySet();
+        Set<InetAddressAndPort> eps = arrivalSamples.keySet();
 
         sb.append("-----------------------------------------------------------------------");
-        for (InetAddress ep : eps)
+        for (InetAddressAndPort ep : eps)
         {
             ArrivalWindow hWnd = arrivalSamples.get(ep);
             sb.append(ep).append(" : ");
@@ -447,7 +477,7 @@ class ArrivalWindow
         }
     }
 
-    synchronized void add(long value, InetAddress ep)
+    synchronized void add(long value, InetAddressAndPort ep)
     {
         assert tLast >= 0;
         if (tLast > 0L)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java b/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java
index 23fae3a..6be31b0 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java
@@ -31,15 +31,18 @@ public interface FailureDetectorMBean
 
     public double getPhiConvictThreshold();
 
-    public String getAllEndpointStates();
+    @Deprecated public String getAllEndpointStates();
+    public String getAllEndpointStatesWithPort();
 
     public String getEndpointState(String address) throws UnknownHostException;
 
-    public Map<String, String> getSimpleStates();
+    @Deprecated public Map<String, String> getSimpleStates();
+    public Map<String, String> getSimpleStatesWithPort();
 
     public int getDownEndpointCount();
 
     public int getUpEndpointCount();
 
-    public TabularData getPhiValues() throws OpenDataException;
+    @Deprecated public TabularData getPhiValues() throws OpenDataException;
+    public TabularData getPhiValuesWithPort() throws OpenDataException;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/GossipDigest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/GossipDigest.java b/src/java/org/apache/cassandra/gms/GossipDigest.java
index 9dfd486..c7e60c4 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigest.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigest.java
@@ -18,12 +18,12 @@
 package org.apache.cassandra.gms;
 
 import java.io.*;
-import java.net.InetAddress;
 
 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.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 
 /**
@@ -34,18 +34,18 @@ public class GossipDigest implements Comparable<GossipDigest>
 {
     public static final IVersionedSerializer<GossipDigest> serializer = new GossipDigestSerializer();
 
-    final InetAddress endpoint;
+    final InetAddressAndPort endpoint;
     final int generation;
     final int maxVersion;
 
-    GossipDigest(InetAddress ep, int gen, int version)
+    GossipDigest(InetAddressAndPort ep, int gen, int version)
     {
         endpoint = ep;
         generation = gen;
         maxVersion = version;
     }
 
-    InetAddress getEndpoint()
+    InetAddressAndPort getEndpoint()
     {
         return endpoint;
     }
@@ -83,14 +83,14 @@ class GossipDigestSerializer implements IVersionedSerializer<GossipDigest>
 {
     public void serialize(GossipDigest gDigest, DataOutputPlus out, int version) throws IOException
     {
-        CompactEndpointSerializationHelper.serialize(gDigest.endpoint, out);
+        CompactEndpointSerializationHelper.instance.serialize(gDigest.endpoint, out, version);
         out.writeInt(gDigest.generation);
         out.writeInt(gDigest.maxVersion);
     }
 
     public GossipDigest deserialize(DataInputPlus in, int version) throws IOException
     {
-        InetAddress endpoint = CompactEndpointSerializationHelper.deserialize(in);
+        InetAddressAndPort endpoint = CompactEndpointSerializationHelper.instance.deserialize(in, version);
         int generation = in.readInt();
         int maxVersion = in.readInt();
         return new GossipDigest(endpoint, generation, maxVersion);
@@ -98,7 +98,7 @@ class GossipDigestSerializer implements IVersionedSerializer<GossipDigest>
 
     public long serializedSize(GossipDigest gDigest, int version)
     {
-        long size = CompactEndpointSerializationHelper.serializedSize(gDigest.endpoint);
+        long size = CompactEndpointSerializationHelper.instance.serializedSize(gDigest.endpoint, version);
         size += TypeSizes.sizeof(gDigest.generation);
         size += TypeSizes.sizeof(gDigest.maxVersion);
         return size;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/GossipDigestAck.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck.java b/src/java/org/apache/cassandra/gms/GossipDigestAck.java
index cf71ae6..a7d5b92 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAck.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAck.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.gms;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -27,6 +26,7 @@ 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.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 
 /**
@@ -38,9 +38,9 @@ public class GossipDigestAck
     public static final IVersionedSerializer<GossipDigestAck> serializer = new GossipDigestAckSerializer();
 
     final List<GossipDigest> gDigestList;
-    final Map<InetAddress, EndpointState> epStateMap;
+    final Map<InetAddressAndPort, EndpointState> epStateMap;
 
-    GossipDigestAck(List<GossipDigest> gDigestList, Map<InetAddress, EndpointState> epStateMap)
+    GossipDigestAck(List<GossipDigest> gDigestList, Map<InetAddressAndPort, EndpointState> epStateMap)
     {
         this.gDigestList = gDigestList;
         this.epStateMap = epStateMap;
@@ -51,7 +51,7 @@ public class GossipDigestAck
         return gDigestList;
     }
 
-    Map<InetAddress, EndpointState> getEndpointStateMap()
+    Map<InetAddressAndPort, EndpointState> getEndpointStateMap()
     {
         return epStateMap;
     }
@@ -63,10 +63,10 @@ class GossipDigestAckSerializer implements IVersionedSerializer<GossipDigestAck>
     {
         GossipDigestSerializationHelper.serialize(gDigestAckMessage.gDigestList, out, version);
         out.writeInt(gDigestAckMessage.epStateMap.size());
-        for (Map.Entry<InetAddress, EndpointState> entry : gDigestAckMessage.epStateMap.entrySet())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : gDigestAckMessage.epStateMap.entrySet())
         {
-            InetAddress ep = entry.getKey();
-            CompactEndpointSerializationHelper.serialize(ep, out);
+            InetAddressAndPort ep = entry.getKey();
+            CompactEndpointSerializationHelper.instance.serialize(ep, out, version);
             EndpointState.serializer.serialize(entry.getValue(), out, version);
         }
     }
@@ -75,11 +75,11 @@ class GossipDigestAckSerializer implements IVersionedSerializer<GossipDigestAck>
     {
         List<GossipDigest> gDigestList = GossipDigestSerializationHelper.deserialize(in, version);
         int size = in.readInt();
-        Map<InetAddress, EndpointState> epStateMap = new HashMap<InetAddress, EndpointState>(size);
+        Map<InetAddressAndPort, EndpointState> epStateMap = new HashMap<InetAddressAndPort, EndpointState>(size);
 
         for (int i = 0; i < size; ++i)
         {
-            InetAddress ep = CompactEndpointSerializationHelper.deserialize(in);
+            InetAddressAndPort ep = CompactEndpointSerializationHelper.instance.deserialize(in, version);
             EndpointState epState = EndpointState.serializer.deserialize(in, version);
             epStateMap.put(ep, epState);
         }
@@ -90,8 +90,8 @@ class GossipDigestAckSerializer implements IVersionedSerializer<GossipDigestAck>
     {
         int size = GossipDigestSerializationHelper.serializedSize(ack.gDigestList, version);
         size += TypeSizes.sizeof(ack.epStateMap.size());
-        for (Map.Entry<InetAddress, EndpointState> entry : ack.epStateMap.entrySet())
-            size += CompactEndpointSerializationHelper.serializedSize(entry.getKey())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : ack.epStateMap.entrySet())
+            size += CompactEndpointSerializationHelper.instance.serializedSize(entry.getKey(), version)
                     + EndpointState.serializer.serializedSize(entry.getValue(), version);
         return size;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
index 9d779fe..a6d1d2b 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.gms;
 
 import java.io.*;
-import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -26,6 +25,7 @@ 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.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 
 /**
@@ -36,14 +36,14 @@ public class GossipDigestAck2
 {
     public static final IVersionedSerializer<GossipDigestAck2> serializer = new GossipDigestAck2Serializer();
 
-    final Map<InetAddress, EndpointState> epStateMap;
+    final Map<InetAddressAndPort, EndpointState> epStateMap;
 
-    GossipDigestAck2(Map<InetAddress, EndpointState> epStateMap)
+    GossipDigestAck2(Map<InetAddressAndPort, EndpointState> epStateMap)
     {
         this.epStateMap = epStateMap;
     }
 
-    Map<InetAddress, EndpointState> getEndpointStateMap()
+    Map<InetAddressAndPort, EndpointState> getEndpointStateMap()
     {
         return epStateMap;
     }
@@ -54,10 +54,10 @@ class GossipDigestAck2Serializer implements IVersionedSerializer<GossipDigestAck
     public void serialize(GossipDigestAck2 ack2, DataOutputPlus out, int version) throws IOException
     {
         out.writeInt(ack2.epStateMap.size());
-        for (Map.Entry<InetAddress, EndpointState> entry : ack2.epStateMap.entrySet())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : ack2.epStateMap.entrySet())
         {
-            InetAddress ep = entry.getKey();
-            CompactEndpointSerializationHelper.serialize(ep, out);
+            InetAddressAndPort ep = entry.getKey();
+            CompactEndpointSerializationHelper.instance.serialize(ep, out, version);
             EndpointState.serializer.serialize(entry.getValue(), out, version);
         }
     }
@@ -65,11 +65,11 @@ class GossipDigestAck2Serializer implements IVersionedSerializer<GossipDigestAck
     public GossipDigestAck2 deserialize(DataInputPlus in, int version) throws IOException
     {
         int size = in.readInt();
-        Map<InetAddress, EndpointState> epStateMap = new HashMap<InetAddress, EndpointState>(size);
+        Map<InetAddressAndPort, EndpointState> epStateMap = new HashMap<>(size);
 
         for (int i = 0; i < size; ++i)
         {
-            InetAddress ep = CompactEndpointSerializationHelper.deserialize(in);
+            InetAddressAndPort ep = CompactEndpointSerializationHelper.instance.deserialize(in, version);
             EndpointState epState = EndpointState.serializer.deserialize(in, version);
             epStateMap.put(ep, epState);
         }
@@ -79,8 +79,8 @@ class GossipDigestAck2Serializer implements IVersionedSerializer<GossipDigestAck
     public long serializedSize(GossipDigestAck2 ack2, int version)
     {
         long size = TypeSizes.sizeof(ack2.epStateMap.size());
-        for (Map.Entry<InetAddress, EndpointState> entry : ack2.epStateMap.entrySet())
-            size += CompactEndpointSerializationHelper.serializedSize(entry.getKey())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : ack2.epStateMap.entrySet())
+            size += CompactEndpointSerializationHelper.instance.serializedSize(entry.getKey(), version)
                     + EndpointState.serializer.serializedSize(entry.getValue(), version);
         return size;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java
index 240bb40..fd5d487 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java
@@ -17,12 +17,12 @@
  */
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
 import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 
@@ -34,7 +34,7 @@ public class GossipDigestAck2VerbHandler implements IVerbHandler<GossipDigestAck
     {
         if (logger.isTraceEnabled())
         {
-            InetAddress from = message.from;
+            InetAddressAndPort from = message.from;
             logger.trace("Received a GossipDigestAck2Message from {}", from);
         }
         if (!Gossiper.instance.isEnabled())
@@ -43,7 +43,7 @@ public class GossipDigestAck2VerbHandler implements IVerbHandler<GossipDigestAck
                 logger.trace("Ignoring GossipDigestAck2Message because gossip is disabled");
             return;
         }
-        Map<InetAddress, EndpointState> remoteEpStateMap = message.payload.getEndpointStateMap();
+        Map<InetAddressAndPort, EndpointState> remoteEpStateMap = message.payload.getEndpointStateMap();
         /* Notify the Failure Detector */
         Gossiper.instance.notifyFailureDetector(remoteEpStateMap);
         Gossiper.instance.applyStateLocally(remoteEpStateMap);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
index d6d9dfb..2a12b7c 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -25,6 +24,7 @@ import java.util.Map;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -36,7 +36,7 @@ public class GossipDigestAckVerbHandler implements IVerbHandler<GossipDigestAck>
 
     public void doVerb(MessageIn<GossipDigestAck> message, int id)
     {
-        InetAddress from = message.from;
+        InetAddressAndPort from = message.from;
         if (logger.isTraceEnabled())
             logger.trace("Received a GossipDigestAckMessage from {}", from);
         if (!Gossiper.instance.isEnabled() && !Gossiper.instance.isInShadowRound())
@@ -48,7 +48,7 @@ public class GossipDigestAckVerbHandler implements IVerbHandler<GossipDigestAck>
 
         GossipDigestAck gDigestAckMessage = message.payload;
         List<GossipDigest> gDigestList = gDigestAckMessage.getGossipDigestList();
-        Map<InetAddress, EndpointState> epStateMap = gDigestAckMessage.getEndpointStateMap();
+        Map<InetAddressAndPort, EndpointState> epStateMap = gDigestAckMessage.getEndpointStateMap();
         logger.trace("Received ack with {} digests and {} states", gDigestList.size(), epStateMap.size());
 
         if (Gossiper.instance.isInShadowRound())
@@ -79,10 +79,10 @@ public class GossipDigestAckVerbHandler implements IVerbHandler<GossipDigestAck>
         }
 
         /* Get the state required to send to this gossipee - construct GossipDigestAck2Message */
-        Map<InetAddress, EndpointState> deltaEpStateMap = new HashMap<InetAddress, EndpointState>();
+        Map<InetAddressAndPort, EndpointState> deltaEpStateMap = new HashMap<InetAddressAndPort, EndpointState>();
         for (GossipDigest gDigest : gDigestList)
         {
-            InetAddress addr = gDigest.getEndpoint();
+            InetAddressAndPort addr = gDigest.getEndpoint();
             EndpointState localEpStatePtr = Gossiper.instance.getStateForVersionBiggerThan(addr, gDigest.getMaxVersion());
             if (localEpStatePtr != null)
                 deltaEpStateMap.put(addr, localEpStatePtr);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
index ddfafc9..9619f4e 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
 import java.util.*;
 
 import com.google.common.collect.Maps;
@@ -26,6 +25,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -37,7 +37,7 @@ public class GossipDigestSynVerbHandler implements IVerbHandler<GossipDigestSyn>
 
     public void doVerb(MessageIn<GossipDigestSyn> message, int id)
     {
-        InetAddress from = message.from;
+        InetAddressAndPort from = message.from;
         if (logger.isTraceEnabled())
             logger.trace("Received a GossipDigestSynMessage from {}", from);
         if (!Gossiper.instance.isEnabled() && !Gossiper.instance.isInShadowRound())
@@ -102,7 +102,7 @@ public class GossipDigestSynVerbHandler implements IVerbHandler<GossipDigestSyn>
         doSort(gDigestList);
 
         List<GossipDigest> deltaGossipDigestList = new ArrayList<GossipDigest>();
-        Map<InetAddress, EndpointState> deltaEpStateMap = new HashMap<InetAddress, EndpointState>();
+        Map<InetAddressAndPort, EndpointState> deltaEpStateMap = new HashMap<InetAddressAndPort, EndpointState>();
         Gossiper.instance.examineGossiper(gDigestList, deltaGossipDigestList, deltaEpStateMap);
         logger.trace("sending {} digests and {} deltas", deltaGossipDigestList.size(), deltaEpStateMap.size());
         MessageOut<GossipDigestAck> gDigestAckMessage = new MessageOut<GossipDigestAck>(MessagingService.Verb.GOSSIP_DIGEST_ACK,
@@ -116,14 +116,14 @@ public class GossipDigestSynVerbHandler implements IVerbHandler<GossipDigestSyn>
     /*
      * First construct a map whose key is the endpoint in the GossipDigest and the value is the
      * GossipDigest itself. Then build a list of version differences i.e difference between the
-     * version in the GossipDigest and the version in the local state for a given InetAddress.
+     * version in the GossipDigest and the version in the local state for a given InetAddressAndPort.
      * Sort this list. Now loop through the sorted list and retrieve the GossipDigest corresponding
      * to the endpoint from the map that was initially constructed.
     */
     private void doSort(List<GossipDigest> gDigestList)
     {
         /* Construct a map of endpoint to GossipDigest. */
-        Map<InetAddress, GossipDigest> epToDigestMap = Maps.newHashMapWithExpectedSize(gDigestList.size());
+        Map<InetAddressAndPort, GossipDigest> epToDigestMap = Maps.newHashMapWithExpectedSize(gDigestList.size());
         for (GossipDigest gDigest : gDigestList)
         {
             epToDigestMap.put(gDigest.getEndpoint(), gDigest);
@@ -136,7 +136,7 @@ public class GossipDigestSynVerbHandler implements IVerbHandler<GossipDigestSyn>
         List<GossipDigest> diffDigests = new ArrayList<GossipDigest>(gDigestList.size());
         for (GossipDigest gDigest : gDigestList)
         {
-            InetAddress ep = gDigest.getEndpoint();
+            InetAddressAndPort ep = gDigest.getEndpoint();
             EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(ep);
             int version = (epState != null) ? Gossiper.instance.getMaxEndpointStateVersion(epState) : 0;
             int diffVersion = Math.abs(version - gDigest.getMaxVersion());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/Gossiper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index e675d92..eb6c500 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -18,12 +18,12 @@
 package org.apache.cassandra.gms;
 
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 import java.util.Map.Entry;
 import java.util.concurrent.*;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
 import javax.management.MBeanServer;
@@ -34,6 +34,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.Uninterruptibles;
 
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.CassandraVersion;
 import org.apache.cassandra.utils.Pair;
 import org.slf4j.Logger;
@@ -99,43 +100,36 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     static final int MAX_GENERATION_DIFFERENCE = 86400 * 365;
     private long fatClientTimeout;
     private final Random random = new Random();
-    private final Comparator<InetAddress> inetcomparator = new Comparator<InetAddress>()
-    {
-        public int compare(InetAddress addr1, InetAddress addr2)
-        {
-            return addr1.getHostAddress().compareTo(addr2.getHostAddress());
-        }
-    };
 
     /* subscribers for interest in EndpointState change */
     private final List<IEndpointStateChangeSubscriber> subscribers = new CopyOnWriteArrayList<IEndpointStateChangeSubscriber>();
 
     /* live member set */
-    private final Set<InetAddress> liveEndpoints = new ConcurrentSkipListSet<InetAddress>(inetcomparator);
+    private final Set<InetAddressAndPort> liveEndpoints = new ConcurrentSkipListSet<>();
 
     /* unreachable member set */
-    private final Map<InetAddress, Long> unreachableEndpoints = new ConcurrentHashMap<InetAddress, Long>();
+    private final Map<InetAddressAndPort, Long> unreachableEndpoints = new ConcurrentHashMap<>();
 
     /* initial seeds for joining the cluster */
     @VisibleForTesting
-    final Set<InetAddress> seeds = new ConcurrentSkipListSet<InetAddress>(inetcomparator);
+    final Set<InetAddressAndPort> seeds = new ConcurrentSkipListSet<>();
 
     /* map where key is the endpoint and value is the state associated with the endpoint */
-    final ConcurrentMap<InetAddress, EndpointState> endpointStateMap = new ConcurrentHashMap<InetAddress, EndpointState>();
+    final ConcurrentMap<InetAddressAndPort, EndpointState> endpointStateMap = new ConcurrentHashMap<>();
 
     /* map where key is endpoint and value is timestamp when this endpoint was removed from
      * gossip. We will ignore any gossip regarding these endpoints for QUARANTINE_DELAY time
      * after removal to prevent nodes from falsely reincarnating during the time when removal
      * gossip gets propagated to all nodes */
-    private final Map<InetAddress, Long> justRemovedEndpoints = new ConcurrentHashMap<InetAddress, Long>();
+    private final Map<InetAddressAndPort, Long> justRemovedEndpoints = new ConcurrentHashMap<>();
 
-    private final Map<InetAddress, Long> expireTimeEndpointMap = new ConcurrentHashMap<InetAddress, Long>();
+    private final Map<InetAddressAndPort, Long> expireTimeEndpointMap = new ConcurrentHashMap<>();
 
     private volatile boolean inShadowRound = false;
     // seeds gathered during shadow round that indicated to be in the shadow round phase as well
-    private final Set<InetAddress> seedsInShadowRound = new ConcurrentSkipListSet<>(inetcomparator);
+    private final Set<InetAddressAndPort> seedsInShadowRound = new ConcurrentSkipListSet<>();
     // endpoint states as gathered during shadow round
-    private final Map<InetAddress, EndpointState> endpointShadowStateMap = new ConcurrentHashMap<>();
+    private final Map<InetAddressAndPort, EndpointState> endpointShadowStateMap = new ConcurrentHashMap<>();
 
     private volatile long lastProcessedMessageAt = System.currentTimeMillis();
 
@@ -151,9 +145,9 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
                 taskLock.lock();
 
                 /* Update the local heartbeat counter. */
-                endpointStateMap.get(FBUtilities.getBroadcastAddress()).getHeartBeatState().updateHeartBeat();
+                endpointStateMap.get(FBUtilities.getBroadcastAddressAndPort()).getHeartBeatState().updateHeartBeat();
                 if (logger.isTraceEnabled())
-                    logger.trace("My heartbeat is now {}", endpointStateMap.get(FBUtilities.getBroadcastAddress()).getHeartBeatState().getHeartBeatVersion());
+                    logger.trace("My heartbeat is now {}", endpointStateMap.get(FBUtilities.getBroadcastAddressAndPort()).getHeartBeatState().getHeartBeatVersion());
                 final List<GossipDigest> gDigests = new ArrayList<GossipDigest>();
                 Gossiper.instance.makeRandomGossipDigest(gDigests);
 
@@ -231,14 +225,24 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
     public boolean seenAnySeed()
     {
-        for (Map.Entry<InetAddress, EndpointState> entry : endpointStateMap.entrySet())
+        for (Map.Entry<InetAddressAndPort, EndpointState> entry : endpointStateMap.entrySet())
         {
             if (seeds.contains(entry.getKey()))
                 return true;
             try
             {
                 VersionedValue internalIp = entry.getValue().getApplicationState(ApplicationState.INTERNAL_IP);
-                if (internalIp != null && seeds.contains(InetAddress.getByName(internalIp.value)))
+                VersionedValue internalIpAndPort = entry.getValue().getApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT);
+                InetAddressAndPort endpoint = null;
+                if (internalIpAndPort != null)
+                {
+                    endpoint = InetAddressAndPort.getByName(internalIpAndPort.value);
+                }
+                else if (internalIp != null)
+                {
+                    endpoint = InetAddressAndPort.getByName(internalIp.value);
+                }
+                if (endpoint != null && seeds.contains(endpoint))
                     return true;
             }
             catch (UnknownHostException e)
@@ -272,18 +276,18 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     /**
      * @return a list of live gossip participants, including fat clients
      */
-    public Set<InetAddress> getLiveMembers()
+    public Set<InetAddressAndPort> getLiveMembers()
     {
-        Set<InetAddress> liveMembers = new HashSet<>(liveEndpoints);
-        if (!liveMembers.contains(FBUtilities.getBroadcastAddress()))
-            liveMembers.add(FBUtilities.getBroadcastAddress());
+        Set<InetAddressAndPort> liveMembers = new HashSet<>(liveEndpoints);
+        if (!liveMembers.contains(FBUtilities.getBroadcastAddressAndPort()))
+            liveMembers.add(FBUtilities.getBroadcastAddressAndPort());
         return liveMembers;
     }
 
     /**
      * @return a list of live ring members.
      */
-    public Set<InetAddress> getLiveTokenOwners()
+    public Set<InetAddressAndPort> getLiveTokenOwners()
     {
         return StorageService.instance.getLiveRingMembers(true);
     }
@@ -291,7 +295,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     /**
      * @return a list of unreachable gossip participants, including fat clients
      */
-    public Set<InetAddress> getUnreachableMembers()
+    public Set<InetAddressAndPort> getUnreachableMembers()
     {
         return unreachableEndpoints.keySet();
     }
@@ -299,10 +303,10 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     /**
      * @return a list of unreachable token owners
      */
-    public Set<InetAddress> getUnreachableTokenOwners()
+    public Set<InetAddressAndPort> getUnreachableTokenOwners()
     {
-        Set<InetAddress> tokenOwners = new HashSet<>();
-        for (InetAddress endpoint : unreachableEndpoints.keySet())
+        Set<InetAddressAndPort> tokenOwners = new HashSet<>();
+        for (InetAddressAndPort endpoint : unreachableEndpoints.keySet())
         {
             if (StorageService.instance.getTokenMetadata().isMember(endpoint))
                 tokenOwners.add(endpoint);
@@ -311,7 +315,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         return tokenOwners;
     }
 
-    public long getEndpointDowntime(InetAddress ep)
+    public long getEndpointDowntime(InetAddressAndPort ep)
     {
         Long downtime = unreachableEndpoints.get(ep);
         if (downtime != null)
@@ -320,14 +324,25 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
             return 0L;
     }
 
-    private boolean isShutdown(InetAddress endpoint)
+    private boolean isShutdown(InetAddressAndPort endpoint)
     {
         EndpointState epState = endpointStateMap.get(endpoint);
         if (epState == null)
+        {
             return false;
-        if (epState.getApplicationState(ApplicationState.STATUS) == null)
-            return false;
-        String value = epState.getApplicationState(ApplicationState.STATUS).value;
+        }
+
+        VersionedValue versionedValue = epState.getApplicationState(ApplicationState.STATUS_WITH_PORT);
+        if (versionedValue == null)
+        {
+            versionedValue = epState.getApplicationState(ApplicationState.STATUS);
+            if (versionedValue == null)
+            {
+                return false;
+            }
+        }
+
+        String value = versionedValue.value;
         String[] pieces = value.split(VersionedValue.DELIMITER_STR, -1);
         assert (pieces.length > 0);
         String state = pieces[0];
@@ -340,7 +355,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      *
      * @param endpoint end point that is convicted.
      */
-    public void convict(InetAddress endpoint, double phi)
+    public void convict(InetAddressAndPort endpoint, double phi)
     {
         EndpointState epState = endpointStateMap.get(endpoint);
         if (epState == null)
@@ -366,11 +381,12 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * This method is used to mark a node as shutdown; that is it gracefully exited on its own and told us about it
      * @param endpoint endpoint that has shut itself down
      */
-    protected void markAsShutdown(InetAddress endpoint)
+    protected void markAsShutdown(InetAddressAndPort endpoint)
     {
         EndpointState epState = endpointStateMap.get(endpoint);
         if (epState == null)
             return;
+        epState.addApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.shutdown(true));
         epState.addApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.shutdown(true));
         epState.addApplicationState(ApplicationState.RPC_READY, StorageService.instance.valueFactory.rpcReady(false));
         epState.getHeartBeatState().forceHighestPossibleVersionUnsafe();
@@ -397,7 +413,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      *
      * @param endpoint endpoint to be removed from the current membership.
      */
-    private void evictFromMembership(InetAddress endpoint)
+    private void evictFromMembership(InetAddressAndPort endpoint)
     {
         unreachableEndpoints.remove(endpoint);
         endpointStateMap.remove(endpoint);
@@ -411,7 +427,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     /**
      * Removes the endpoint from Gossip but retains endpoint state
      */
-    public void removeEndpoint(InetAddress endpoint)
+    public void removeEndpoint(InetAddressAndPort endpoint)
     {
         // do subscribers first so anything in the subscriber that depends on gossiper state won't get confused
         for (IEndpointStateChangeSubscriber subscriber : subscribers)
@@ -438,7 +454,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      *
      * @param endpoint
      */
-    private void quarantineEndpoint(InetAddress endpoint)
+    private void quarantineEndpoint(InetAddressAndPort endpoint)
     {
         quarantineEndpoint(endpoint, System.currentTimeMillis());
     }
@@ -449,7 +465,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * @param endpoint
      * @param quarantineExpiration
      */
-    private void quarantineEndpoint(InetAddress endpoint, long quarantineExpiration)
+    private void quarantineEndpoint(InetAddressAndPort endpoint, long quarantineExpiration)
     {
         justRemovedEndpoints.put(endpoint, quarantineExpiration);
     }
@@ -458,7 +474,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * Quarantine endpoint specifically for replacement purposes.
      * @param endpoint
      */
-    public void replacementQuarantine(InetAddress endpoint)
+    public void replacementQuarantine(InetAddressAndPort endpoint)
     {
         // remember, quarantineEndpoint will effectively already add QUARANTINE_DELAY, so this is 2x
         logger.debug("");
@@ -471,7 +487,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      *
      * @param endpoint The endpoint that has been replaced
      */
-    public void replacedEndpoint(InetAddress endpoint)
+    public void replacedEndpoint(InetAddressAndPort endpoint)
     {
         removeEndpoint(endpoint);
         evictFromMembership(endpoint);
@@ -491,9 +507,9 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         int maxVersion = 0;
 
         // local epstate will be part of endpointStateMap
-        List<InetAddress> endpoints = new ArrayList<InetAddress>(endpointStateMap.keySet());
+        List<InetAddressAndPort> endpoints = new ArrayList<>(endpointStateMap.keySet());
         Collections.shuffle(endpoints, random);
-        for (InetAddress endpoint : endpoints)
+        for (InetAddressAndPort endpoint : endpoints)
         {
             epState = endpointStateMap.get(endpoint);
             if (epState != null)
@@ -524,7 +540,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * @param hostId      - the ID of the host being removed
      * @param localHostId - my own host ID for replication coordination
      */
-    public void advertiseRemoving(InetAddress endpoint, UUID hostId, UUID localHostId)
+    public void advertiseRemoving(InetAddressAndPort endpoint, UUID hostId, UUID localHostId)
     {
         EndpointState epState = endpointStateMap.get(endpoint);
         // remember this node's generation
@@ -541,6 +557,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         epState.updateTimestamp(); // make sure we don't evict it too soon
         epState.getHeartBeatState().forceNewerGenerationUnsafe();
         Map<ApplicationState, VersionedValue> states = new EnumMap<>(ApplicationState.class);
+        states.put(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.removingNonlocal(hostId));
         states.put(ApplicationState.STATUS, StorageService.instance.valueFactory.removingNonlocal(hostId));
         states.put(ApplicationState.REMOVAL_COORDINATOR, StorageService.instance.valueFactory.removalCoordinator(localHostId));
         epState.addApplicationStates(states);
@@ -554,12 +571,13 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * @param endpoint
      * @param hostId
      */
-    public void advertiseTokenRemoved(InetAddress endpoint, UUID hostId)
+    public void advertiseTokenRemoved(InetAddressAndPort endpoint, UUID hostId)
     {
         EndpointState epState = endpointStateMap.get(endpoint);
         epState.updateTimestamp(); // make sure we don't evict it too soon
         epState.getHeartBeatState().forceNewerGenerationUnsafe();
         long expireTime = computeExpireTime();
+        epState.addApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.removedNonlocal(hostId, expireTime));
         epState.addApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.removedNonlocal(hostId, expireTime));
         logger.info("Completing removal of {}", endpoint);
         addExpireTimeForEndpoint(endpoint, expireTime);
@@ -584,7 +602,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      */
     public void assassinateEndpoint(String address) throws UnknownHostException
     {
-        InetAddress endpoint = InetAddress.getByName(address);
+        InetAddressAndPort endpoint = InetAddressAndPort.getByName(address);
         EndpointState epState = endpointStateMap.get(endpoint);
         Collection<Token> tokens = null;
         logger.warn("Assassinating {} via gossip", endpoint);
@@ -624,18 +642,20 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         }
 
         // do not pass go, do not collect 200 dollars, just gtfo
-        epState.addApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.left(tokens, computeExpireTime()));
+        long expireTime = computeExpireTime();
+        epState.addApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.left(tokens, expireTime));
+        epState.addApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.left(tokens, expireTime));
         handleMajorStateChange(endpoint, epState);
         Uninterruptibles.sleepUninterruptibly(intervalInMillis * 4, TimeUnit.MILLISECONDS);
         logger.warn("Finished assassinating {}", endpoint);
     }
 
-    public boolean isKnownEndpoint(InetAddress endpoint)
+    public boolean isKnownEndpoint(InetAddressAndPort endpoint)
     {
         return endpointStateMap.containsKey(endpoint);
     }
 
-    public int getCurrentGenerationNumber(InetAddress endpoint)
+    public int getCurrentGenerationNumber(InetAddressAndPort endpoint)
     {
         return endpointStateMap.get(endpoint).getHeartBeatState().getGeneration();
     }
@@ -647,16 +667,16 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * @param epSet   a set of endpoint from which a random endpoint is chosen.
      * @return true if the chosen endpoint is also a seed.
      */
-    private boolean sendGossip(MessageOut<GossipDigestSyn> message, Set<InetAddress> epSet)
+    private boolean sendGossip(MessageOut<GossipDigestSyn> message, Set<InetAddressAndPort> epSet)
     {
-        List<InetAddress> liveEndpoints = ImmutableList.copyOf(epSet);
+        List<InetAddressAndPort> liveEndpoints = ImmutableList.copyOf(epSet);
 
         int size = liveEndpoints.size();
         if (size < 1)
             return false;
         /* Generate a random number from 0 -> size */
         int index = (size == 1) ? 0 : random.nextInt(size);
-        InetAddress to = liveEndpoints.get(index);
+        InetAddressAndPort to = liveEndpoints.get(index);
         if (logger.isTraceEnabled())
             logger.trace("Sending a GossipDigestSyn to {} ...", to);
         if (firstSynSendAt == 0)
@@ -695,7 +715,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         int size = seeds.size();
         if (size > 0)
         {
-            if (size == 1 && seeds.contains(FBUtilities.getBroadcastAddress()))
+            if (size == 1 && seeds.contains(FBUtilities.getBroadcastAddressAndPort()))
             {
                 return;
             }
@@ -715,7 +735,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         }
     }
 
-    public boolean isGossipOnlyMember(InetAddress endpoint)
+    public boolean isGossipOnlyMember(InetAddressAndPort endpoint)
     {
         EndpointState epState = endpointStateMap.get(endpoint);
         if (epState == null)
@@ -740,8 +760,8 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * @param epStates - endpoint states in the cluster
      * @return true if it is safe to start the node, false otherwise
      */
-    public boolean isSafeForStartup(InetAddress endpoint, UUID localHostUUID, boolean isBootstrapping,
-                                    Map<InetAddress, EndpointState> epStates)
+    public boolean isSafeForStartup(InetAddressAndPort endpoint, UUID localHostUUID, boolean isBootstrapping,
+                                    Map<InetAddressAndPort, EndpointState> epStates)
     {
         EndpointState epState = epStates.get(endpoint);
         // if there's no previous state, or the node was previously removed from the cluster, we're good
@@ -792,10 +812,10 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
             }
         }
 
-        Set<InetAddress> eps = endpointStateMap.keySet();
-        for (InetAddress endpoint : eps)
+        Set<InetAddressAndPort> eps = endpointStateMap.keySet();
+        for (InetAddressAndPort endpoint : eps)
         {
-            if (endpoint.equals(FBUtilities.getBroadcastAddress()))
+            if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
                 continue;
 
             FailureDetector.instance.interpret(endpoint);
@@ -829,7 +849,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
         if (!justRemovedEndpoints.isEmpty())
         {
-            for (Entry<InetAddress, Long> entry : justRemovedEndpoints.entrySet())
+            for (Entry<InetAddressAndPort, Long> entry : justRemovedEndpoints.entrySet())
             {
                 if ((now - entry.getValue()) > QUARANTINE_DELAY)
                 {
@@ -841,34 +861,34 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         }
     }
 
-    protected long getExpireTimeForEndpoint(InetAddress endpoint)
+    protected long getExpireTimeForEndpoint(InetAddressAndPort endpoint)
     {
         /* default expireTime is aVeryLongTime */
         Long storedTime = expireTimeEndpointMap.get(endpoint);
         return storedTime == null ? computeExpireTime() : storedTime;
     }
 
-    public EndpointState getEndpointStateForEndpoint(InetAddress ep)
+    public EndpointState getEndpointStateForEndpoint(InetAddressAndPort ep)
     {
         return endpointStateMap.get(ep);
     }
 
-    public Set<Entry<InetAddress, EndpointState>> getEndpointStates()
+    public Set<Entry<InetAddressAndPort, EndpointState>> getEndpointStates()
     {
         return endpointStateMap.entrySet();
     }
 
-    public UUID getHostId(InetAddress endpoint)
+    public UUID getHostId(InetAddressAndPort endpoint)
     {
         return getHostId(endpoint, endpointStateMap);
     }
 
-    public UUID getHostId(InetAddress endpoint, Map<InetAddress, EndpointState> epStates)
+    public UUID getHostId(InetAddressAndPort endpoint, Map<InetAddressAndPort, EndpointState> epStates)
     {
         return UUID.fromString(epStates.get(endpoint).getApplicationState(ApplicationState.HOST_ID).value);
     }
 
-    EndpointState getStateForVersionBiggerThan(InetAddress forEndpoint, int version)
+    EndpointState getStateForVersionBiggerThan(InetAddressAndPort forEndpoint, int version)
     {
         EndpointState epState = endpointStateMap.get(forEndpoint);
         EndpointState reqdEndpointState = null;
@@ -919,7 +939,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     /**
      * determine which endpoint started up earlier
      */
-    public int compareEndpointStartup(InetAddress addr1, InetAddress addr2)
+    public int compareEndpointStartup(InetAddressAndPort addr1, InetAddressAndPort addr2)
     {
         EndpointState ep1 = getEndpointStateForEndpoint(addr1);
         EndpointState ep2 = getEndpointStateForEndpoint(addr2);
@@ -927,15 +947,15 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         return ep1.getHeartBeatState().getGeneration() - ep2.getHeartBeatState().getGeneration();
     }
 
-    void notifyFailureDetector(Map<InetAddress, EndpointState> remoteEpStateMap)
+    void notifyFailureDetector(Map<InetAddressAndPort, EndpointState> remoteEpStateMap)
     {
-        for (Entry<InetAddress, EndpointState> entry : remoteEpStateMap.entrySet())
+        for (Entry<InetAddressAndPort, EndpointState> entry : remoteEpStateMap.entrySet())
         {
             notifyFailureDetector(entry.getKey(), entry.getValue());
         }
     }
 
-    void notifyFailureDetector(InetAddress endpoint, EndpointState remoteEndpointState)
+    void notifyFailureDetector(InetAddressAndPort endpoint, EndpointState remoteEndpointState)
     {
         EndpointState localEndpointState = endpointStateMap.get(endpoint);
         /*
@@ -976,7 +996,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
     }
 
-    private void markAlive(final InetAddress addr, final EndpointState localState)
+    private void markAlive(final InetAddressAndPort addr, final EndpointState localState)
     {
         localState.markDead();
 
@@ -999,7 +1019,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     @VisibleForTesting
-    public void realMarkAlive(final InetAddress addr, final EndpointState localState)
+    public void realMarkAlive(final InetAddressAndPort addr, final EndpointState localState)
     {
         if (logger.isTraceEnabled())
             logger.trace("marking as alive {}", addr);
@@ -1017,7 +1037,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     @VisibleForTesting
-    public void markDead(InetAddress addr, EndpointState localState)
+    public void markDead(InetAddressAndPort addr, EndpointState localState)
     {
         if (logger.isTraceEnabled())
             logger.trace("marking as down {}", addr);
@@ -1037,7 +1057,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * @param ep      endpoint
      * @param epState EndpointState for the endpoint
      */
-    private void handleMajorStateChange(InetAddress ep, EndpointState epState)
+    private void handleMajorStateChange(InetAddressAndPort ep, EndpointState epState)
     {
         EndpointState localEpState = endpointStateMap.get(ep);
         if (!isDeadState(epState))
@@ -1071,7 +1091,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
             markAsShutdown(ep);
     }
 
-    public boolean isAlive(InetAddress endpoint)
+    public boolean isAlive(InetAddressAndPort endpoint)
     {
         EndpointState epState = getEndpointStateForEndpoint(endpoint);
         if (epState == null)
@@ -1099,21 +1119,33 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
     private static String getGossipStatus(EndpointState epState)
     {
-        if (epState == null || epState.getApplicationState(ApplicationState.STATUS) == null)
+        if (epState == null)
+        {
             return "";
+        }
+
+        VersionedValue versionedValue = epState.getApplicationState(ApplicationState.STATUS_WITH_PORT);
+        if (versionedValue == null)
+        {
+            versionedValue = epState.getApplicationState(ApplicationState.STATUS);
+            if (versionedValue == null)
+            {
+                return "";
+            }
+        }
 
-        String value = epState.getApplicationState(ApplicationState.STATUS).value;
+        String value = versionedValue.value;
         String[] pieces = value.split(VersionedValue.DELIMITER_STR, -1);
         assert (pieces.length > 0);
         return pieces[0];
     }
 
-    void applyStateLocally(Map<InetAddress, EndpointState> epStateMap)
+    void applyStateLocally(Map<InetAddressAndPort, EndpointState> epStateMap)
     {
-        for (Entry<InetAddress, EndpointState> entry : epStateMap.entrySet())
+        for (Entry<InetAddressAndPort, EndpointState> entry : epStateMap.entrySet())
         {
-            InetAddress ep = entry.getKey();
-            if ( ep.equals(FBUtilities.getBroadcastAddress()) && !isInShadowRound())
+            InetAddressAndPort ep = entry.getKey();
+            if ( ep.equals(FBUtilities.getBroadcastAddressAndPort()) && !isInShadowRound())
                 continue;
             if (justRemovedEndpoints.containsKey(ep))
             {
@@ -1181,7 +1213,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         }
     }
 
-    private void applyNewStates(InetAddress addr, EndpointState localState, EndpointState remoteState)
+    private void applyNewStates(InetAddressAndPort addr, EndpointState localState, EndpointState remoteState)
     {
         // don't assert here, since if the node restarts the version will go back to zero
         int oldVersion = localState.getHeartBeatState().getHeartBeatVersion();
@@ -1194,12 +1226,27 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         assert remoteState.getHeartBeatState().getGeneration() == localState.getHeartBeatState().getGeneration();
         localState.addApplicationStates(remoteStates);
 
-        for (Entry<ApplicationState, VersionedValue> remoteEntry : remoteStates)
+        //Filter out pre-4.0 versions of data for more complete 4.0 versions
+        Set<Entry<ApplicationState, VersionedValue>> filtered = remoteStates.stream().filter(entry -> {
+           switch (entry.getKey())
+           {
+               case INTERNAL_IP:
+                    return remoteState.getApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT) == null;
+               case STATUS:
+                   return remoteState.getApplicationState(ApplicationState.STATUS_WITH_PORT) == null;
+               case RPC_ADDRESS:
+                   return remoteState.getApplicationState(ApplicationState.NATIVE_ADDRESS_AND_PORT) == null;
+               default:
+                   return true;
+           }
+        }).collect(Collectors.toSet());
+
+        for (Entry<ApplicationState, VersionedValue> remoteEntry : filtered)
             doOnChangeNotifications(addr, remoteEntry.getKey(), remoteEntry.getValue());
     }
 
     // notify that a local application state is going to change (doesn't get triggered for remote changes)
-    private void doBeforeChangeNotifications(InetAddress addr, EndpointState epState, ApplicationState apState, VersionedValue newValue)
+    private void doBeforeChangeNotifications(InetAddressAndPort addr, EndpointState epState, ApplicationState apState, VersionedValue newValue)
     {
         for (IEndpointStateChangeSubscriber subscriber : subscribers)
         {
@@ -1208,7 +1255,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     // notify that an application state has changed
-    private void doOnChangeNotifications(InetAddress addr, ApplicationState state, VersionedValue value)
+    private void doOnChangeNotifications(InetAddressAndPort addr, ApplicationState state, VersionedValue value)
     {
         for (IEndpointStateChangeSubscriber subscriber : subscribers)
         {
@@ -1226,7 +1273,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     /* Send all the data with version greater than maxRemoteVersion */
-    private void sendAll(GossipDigest gDigest, Map<InetAddress, EndpointState> deltaEpStateMap, int maxRemoteVersion)
+    private void sendAll(GossipDigest gDigest, Map<InetAddressAndPort, EndpointState> deltaEpStateMap, int maxRemoteVersion)
     {
         EndpointState localEpStatePtr = getStateForVersionBiggerThan(gDigest.getEndpoint(), maxRemoteVersion);
         if (localEpStatePtr != null)
@@ -1237,7 +1284,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         This method is used to figure the state that the Gossiper has but Gossipee doesn't. The delta digests
         and the delta state are built up.
     */
-    void examineGossiper(List<GossipDigest> gDigestList, List<GossipDigest> deltaGossipDigestList, Map<InetAddress, EndpointState> deltaEpStateMap)
+    void examineGossiper(List<GossipDigest> gDigestList, List<GossipDigest> deltaGossipDigestList, Map<InetAddressAndPort, EndpointState> deltaEpStateMap)
     {
         if (gDigestList.size() == 0)
         {
@@ -1245,7 +1292,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
               If this is happening then the node is attempting shadow gossip, and we should reply with everything we know.
             */
             logger.debug("Shadow request received, adding all states");
-            for (Map.Entry<InetAddress, EndpointState> entry : endpointStateMap.entrySet())
+            for (Map.Entry<InetAddressAndPort, EndpointState> entry : endpointStateMap.entrySet())
             {
                 gDigestList.add(new GossipDigest(entry.getKey(), 0, 0));
             }
@@ -1320,7 +1367,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         buildSeedsList();
         /* initialize the heartbeat state for this localEndpoint */
         maybeInitializeLocalState(generationNbr);
-        EndpointState localState = endpointStateMap.get(FBUtilities.getBroadcastAddress());
+        EndpointState localState = endpointStateMap.get(FBUtilities.getBroadcastAddressAndPort());
         localState.addApplicationStates(preloadLocalStates);
 
         //notify snitches that Gossiper is about to start
@@ -1345,14 +1392,14 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * </ul>
      *
      * Method is synchronized, as we use an in-progress flag to indicate that shadow round must be cleared
-     * again by calling {@link Gossiper#maybeFinishShadowRound(InetAddress, boolean, Map)}. This will update
+     * again by calling {@link Gossiper#maybeFinishShadowRound(InetAddressAndPort, boolean, Map)}. This will update
      * {@link Gossiper#endpointShadowStateMap} with received values, in order to return an immutable copy to the
      * caller of {@link Gossiper#doShadowRound()}. Therefor only a single shadow round execution is permitted at
      * the same time.
      *
      * @return endpoint states gathered during shadow round or empty map
      */
-    public synchronized Map<InetAddress, EndpointState> doShadowRound()
+    public synchronized Map<InetAddressAndPort, EndpointState> doShadowRound()
     {
         buildSeedsList();
         // it may be that the local address is the only entry in the seed
@@ -1381,7 +1428,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
                 { // CASSANDRA-8072, retry at the beginning and every 5 seconds
                     logger.trace("Sending shadow round GOSSIP DIGEST SYN to seeds {}", seeds);
 
-                    for (InetAddress seed : seeds)
+                    for (InetAddressAndPort seed : seeds)
                         MessagingService.instance().sendOneWay(message, seed);
                 }
 
@@ -1393,8 +1440,8 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
                 if (slept > StorageService.RING_DELAY)
                 {
                     // if we don't consider ourself to be a seed, fail out
-                    if (!DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()))
-                        throw new RuntimeException("Unable to gossip with any seeds");
+                    if (!DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddressAndPort()))
+                        throw new RuntimeException("Unable to gossip with any seeds " + DatabaseDescriptor.getSeeds() + " and " + FBUtilities.getBroadcastAddressAndPort());
 
                     logger.warn("Unable to gossip with any seeds but continuing since node is in its own seed list");
                     inShadowRound = false;
@@ -1413,9 +1460,9 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     @VisibleForTesting
     void buildSeedsList()
     {
-        for (InetAddress seed : DatabaseDescriptor.getSeeds())
+        for (InetAddressAndPort seed : DatabaseDescriptor.getSeeds())
         {
-            if (seed.equals(FBUtilities.getBroadcastAddress()))
+            if (seed.equals(FBUtilities.getBroadcastAddressAndPort()))
                 continue;
             seeds.add(seed);
         }
@@ -1427,12 +1474,12 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         HeartBeatState hbState = new HeartBeatState(generationNbr);
         EndpointState localState = new EndpointState(hbState);
         localState.markAlive();
-        endpointStateMap.putIfAbsent(FBUtilities.getBroadcastAddress(), localState);
+        endpointStateMap.putIfAbsent(FBUtilities.getBroadcastAddressAndPort(), localState);
     }
 
     public void forceNewerGeneration()
     {
-        EndpointState epstate = endpointStateMap.get(FBUtilities.getBroadcastAddress());
+        EndpointState epstate = endpointStateMap.get(FBUtilities.getBroadcastAddressAndPort());
         epstate.getHeartBeatState().forceNewerGenerationUnsafe();
     }
 
@@ -1440,9 +1487,9 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     /**
      * Add an endpoint we knew about previously, but whose state is unknown
      */
-    public void addSavedEndpoint(InetAddress ep)
+    public void addSavedEndpoint(InetAddressAndPort ep)
     {
-        if (ep.equals(FBUtilities.getBroadcastAddress()))
+        if (ep.equals(FBUtilities.getBroadcastAddressAndPort()))
         {
             logger.debug("Attempt to add self as saved endpoint");
             return;
@@ -1470,8 +1517,8 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     private void addLocalApplicationStateInternal(ApplicationState state, VersionedValue value)
     {
         assert taskLock.isHeldByCurrentThread();
-        EndpointState epState = endpointStateMap.get(FBUtilities.getBroadcastAddress());
-        InetAddress epAddr = FBUtilities.getBroadcastAddress();
+        EndpointState epState = endpointStateMap.get(FBUtilities.getBroadcastAddressAndPort());
+        InetAddressAndPort epAddr = FBUtilities.getBroadcastAddressAndPort();
         assert epState != null;
         // Fire "before change" notifications:
         doBeforeChangeNotifications(epAddr, epState, state, value);
@@ -1508,13 +1555,14 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
     public void stop()
     {
-        EndpointState mystate = endpointStateMap.get(FBUtilities.getBroadcastAddress());
+        EndpointState mystate = endpointStateMap.get(FBUtilities.getBroadcastAddressAndPort());
         if (mystate != null && !isSilentShutdownState(mystate) && StorageService.instance.isJoined())
         {
             logger.info("Announcing shutdown");
+            addLocalApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.shutdown(true));
             addLocalApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.shutdown(true));
             MessageOut message = new MessageOut(MessagingService.Verb.GOSSIP_SHUTDOWN);
-            for (InetAddress ep : liveEndpoints)
+            for (InetAddressAndPort ep : liveEndpoints)
                 MessagingService.instance().sendOneWay(message, ep);
             Uninterruptibles.sleepUninterruptibly(Integer.getInteger("cassandra.shutdown_announce_in_ms", 2000), TimeUnit.MILLISECONDS);
         }
@@ -1529,7 +1577,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         return (scheduledGossipTask != null) && (!scheduledGossipTask.isCancelled());
     }
 
-    protected void maybeFinishShadowRound(InetAddress respondent, boolean isInShadowRound, Map<InetAddress, EndpointState> epStateMap)
+    protected void maybeFinishShadowRound(InetAddressAndPort respondent, boolean isInShadowRound, Map<InetAddressAndPort, EndpointState> epStateMap)
     {
         if (inShadowRound)
         {
@@ -1565,7 +1613,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     @VisibleForTesting
-    public void initializeNodeUnsafe(InetAddress addr, UUID uuid, int generationNbr)
+    public void initializeNodeUnsafe(InetAddressAndPort addr, UUID uuid, int generationNbr)
     {
         HeartBeatState hbState = new HeartBeatState(generationNbr);
         EndpointState newState = new EndpointState(hbState);
@@ -1581,7 +1629,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     @VisibleForTesting
-    public void injectApplicationState(InetAddress endpoint, ApplicationState state, VersionedValue value)
+    public void injectApplicationState(InetAddressAndPort endpoint, ApplicationState state, VersionedValue value)
     {
         EndpointState localState = endpointStateMap.get(endpoint);
         localState.addApplicationState(state, value);
@@ -1589,15 +1637,15 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
     public long getEndpointDowntime(String address) throws UnknownHostException
     {
-        return getEndpointDowntime(InetAddress.getByName(address));
+        return getEndpointDowntime(InetAddressAndPort.getByName(address));
     }
 
     public int getCurrentGenerationNumber(String address) throws UnknownHostException
     {
-        return getCurrentGenerationNumber(InetAddress.getByName(address));
+        return getCurrentGenerationNumber(InetAddressAndPort.getByName(address));
     }
 
-    public void addExpireTimeForEndpoint(InetAddress endpoint, long expireTime)
+    public void addExpireTimeForEndpoint(InetAddressAndPort endpoint, long expireTime)
     {
         if (logger.isDebugEnabled())
         {
@@ -1612,14 +1660,14 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     @Nullable
-    public CassandraVersion getReleaseVersion(InetAddress ep)
+    public CassandraVersion getReleaseVersion(InetAddressAndPort ep)
     {
         EndpointState state = getEndpointStateForEndpoint(ep);
         return state != null ? state.getReleaseVersion() : null;
     }
 
     @Nullable
-    public UUID getSchemaVersion(InetAddress ep)
+    public UUID getSchemaVersion(InetAddressAndPort ep)
     {
         EndpointState state = getEndpointStateForEndpoint(ep);
         return state != null ? state.getSchemaVersion() : null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java b/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java
index 1bfd678..dc81650 100644
--- a/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java
+++ b/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * This is called by an instance of the IEndpointStateChangePublisher to notify
@@ -36,17 +36,17 @@ public interface IEndpointStateChangeSubscriber
      * @param endpoint endpoint for which the state change occurred.
      * @param epState  state that actually changed for the above endpoint.
      */
-    public void onJoin(InetAddress endpoint, EndpointState epState);
+    public void onJoin(InetAddressAndPort endpoint, EndpointState epState);
     
-    public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue);
+    public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue);
 
-    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value);
+    public void onChange(InetAddressAndPort endpoint, ApplicationState state, VersionedValue value);
 
-    public void onAlive(InetAddress endpoint, EndpointState state);
+    public void onAlive(InetAddressAndPort endpoint, EndpointState state);
 
-    public void onDead(InetAddress endpoint, EndpointState state);
+    public void onDead(InetAddressAndPort endpoint, EndpointState state);
 
-    public void onRemove(InetAddress endpoint);
+    public void onRemove(InetAddressAndPort endpoint);
 
     /**
      * Called whenever a node is restarted.
@@ -54,5 +54,5 @@ public interface IEndpointStateChangeSubscriber
      * previously marked down. It will have only if {@code state.isAlive() == false}
      * as {@code state} is from before the restarted node is marked up.
      */
-    public void onRestart(InetAddress endpoint, EndpointState state);
+    public void onRestart(InetAddressAndPort endpoint, EndpointState state);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/IFailureDetectionEventListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/IFailureDetectionEventListener.java b/src/java/org/apache/cassandra/gms/IFailureDetectionEventListener.java
index 8b274b6..4e0c663 100644
--- a/src/java/org/apache/cassandra/gms/IFailureDetectionEventListener.java
+++ b/src/java/org/apache/cassandra/gms/IFailureDetectionEventListener.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * Implemented by the Gossiper to convict an endpoint
@@ -33,5 +33,5 @@ public interface IFailureDetectionEventListener
      * @param ep  endpoint to be convicted
      * @param phi the value of phi with with ep was convicted
      */
-    public void convict(InetAddress ep, double phi);
+    public void convict(InetAddressAndPort ep, double phi);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/IFailureDetector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/IFailureDetector.java b/src/java/org/apache/cassandra/gms/IFailureDetector.java
index a860c7c..62fc97d 100644
--- a/src/java/org/apache/cassandra/gms/IFailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/IFailureDetector.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.gms;
 
-import java.net.InetAddress;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 /**
  * An interface that provides an application with the ability
@@ -35,7 +35,7 @@ public interface IFailureDetector
      * @param ep endpoint in question.
      * @return true if UP and false if DOWN.
      */
-    public boolean isAlive(InetAddress ep);
+    public boolean isAlive(InetAddressAndPort ep);
 
     /**
      * This method is invoked by any entity wanting to interrogate the status of an endpoint.
@@ -44,7 +44,7 @@ public interface IFailureDetector
      *
      * param ep endpoint for which we interpret the inter arrival times.
      */
-    public void interpret(InetAddress ep);
+    public void interpret(InetAddressAndPort ep);
 
     /**
      * This method is invoked by the receiver of the heartbeat. In our case it would be
@@ -53,17 +53,17 @@ public interface IFailureDetector
      *
      * param ep endpoint being reported.
      */
-    public void report(InetAddress ep);
+    public void report(InetAddressAndPort ep);
 
     /**
      * remove endpoint from failure detector
      */
-    public void remove(InetAddress ep);
+    public void remove(InetAddressAndPort ep);
 
     /**
      * force conviction of endpoint in the failure detector
      */
-    public void forceConviction(InetAddress ep);
+    public void forceConviction(InetAddressAndPort ep);
 
     /**
      * Register interest for Failure Detector events.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59b5b6be/src/java/org/apache/cassandra/gms/VersionedValue.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/VersionedValue.java b/src/java/org/apache/cassandra/gms/VersionedValue.java
index d9c8d0b..691f544 100644
--- a/src/java/org/apache/cassandra/gms/VersionedValue.java
+++ b/src/java/org/apache/cassandra/gms/VersionedValue.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.commons.lang3.StringUtils;
@@ -133,11 +134,17 @@ public class VersionedValue implements Comparable<VersionedValue>
             return new VersionedValue(value.value);
         }
 
+        @Deprecated
         public VersionedValue bootReplacing(InetAddress oldNode)
         {
             return new VersionedValue(versionString(VersionedValue.STATUS_BOOTSTRAPPING_REPLACE, oldNode.getHostAddress()));
         }
 
+        public VersionedValue bootReplacingWithPort(InetAddressAndPort oldNode)
+        {
+            return new VersionedValue(versionString(VersionedValue.STATUS_BOOTSTRAPPING_REPLACE, oldNode.toString()));
+        }
+
         public VersionedValue bootstrapping(Collection<Token> tokens)
         {
             return new VersionedValue(versionString(VersionedValue.STATUS_BOOTSTRAPPING,
@@ -248,6 +255,11 @@ public class VersionedValue implements Comparable<VersionedValue>
             return new VersionedValue(endpoint.getHostAddress());
         }
 
+        public VersionedValue nativeaddressAndPort(InetAddressAndPort address)
+        {
+            return new VersionedValue(address.toString());
+        }
+
         public VersionedValue releaseVersion()
         {
             return new VersionedValue(FBUtilities.getReleaseVersionString());
@@ -263,6 +275,11 @@ public class VersionedValue implements Comparable<VersionedValue>
             return new VersionedValue(private_ip);
         }
 
+        public VersionedValue internalAddressAndPort(InetAddressAndPort address)
+        {
+            return new VersionedValue(address.toString());
+        }
+
         public VersionedValue severity(double value)
         {
             return new VersionedValue(String.valueOf(value));


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