You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2016/08/12 23:35:46 UTC

[1/3] cassandra git commit: Only set broadcast_rpc_address on Ec2MultiRegionSnitch if it's not set

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.9 d052c9048 -> d597910b2


Only set broadcast_rpc_address on Ec2MultiRegionSnitch if it's not set

Patch by Paulo Motta; reviewed by Tyler Hobbs for CASSANDRA-11356


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

Branch: refs/heads/cassandra-3.9
Commit: 91f7387e1f785b18321777311a5c3416af0663c2
Parents: e86d531
Author: Paulo Motta <pa...@gmail.com>
Authored: Tue Aug 2 12:42:20 2016 -0300
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Fri Aug 12 18:25:19 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  6 ++++
 .../cassandra/config/DatabaseDescriptor.java    |  6 ++--
 .../cassandra/locator/Ec2MultiRegionSnitch.java |  6 +++-
 .../cassandra/service/StorageService.java       |  4 +--
 .../org/apache/cassandra/transport/Server.java  |  2 +-
 .../org/apache/cassandra/utils/FBUtilities.java | 20 ++++++++++++
 .../apache/cassandra/utils/FBUtilitiesTest.java | 32 ++++++++++++++++++++
 8 files changed, 71 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/91f7387e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ddc6720..394598a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.8
+ * Only set broadcast_rpc_address on Ec2MultiRegionSnitch if it's not set (CASSANDRA-11357)
  * Update StorageProxy range metrics for timeouts, failures and unavailables (CASSANDRA-9507)
  * Add Sigar to classes included in clientutil.jar (CASSANDRA-11635)
  * Add decay to histograms and timers used for metrics (CASSANDRA-11752)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91f7387e/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index a3ba0dd..f0712eb 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -21,6 +21,12 @@ New features
     - JSON timestamps are now in UTC and contain the timezone information, see
       CASSANDRA-11137 for more details.
 
+Upgrading
+---------
+    - Ec2MultiRegionSnitch will no longer automatically set broadcast_rpc_address
+      to the public instance IP if this property is defined on cassandra.yaml.
+
+
 2.2.6
 =====
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91f7387e/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 6e46725..75f80b9 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -174,7 +174,7 @@ public class DatabaseDescriptor
     }
 
     @VisibleForTesting
-    static void applyAddressConfig(Config config) throws ConfigurationException
+    public static void applyAddressConfig(Config config) throws ConfigurationException
     {
         listenAddress = null;
         rpcAddress = null;
@@ -266,7 +266,6 @@ public class DatabaseDescriptor
             if (rpcAddress.isAnyLocalAddress())
                 throw new ConfigurationException("If rpc_address is set to a wildcard address (" + config.rpc_address + "), then " +
                                                  "you must set broadcast_rpc_address to a value other than " + config.rpc_address, false);
-            broadcastRpcAddress = rpcAddress;
         }
     }
 
@@ -1269,6 +1268,9 @@ public class DatabaseDescriptor
         broadcastRpcAddress = broadcastRPCAddr;
     }
 
+    /**
+     * May be null, please use {@link FBUtilities#getBroadcastRpcAddress()} instead.
+     */
     public static InetAddress getBroadcastRpcAddress()
     {
         return broadcastRpcAddress;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91f7387e/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 ab1b5d0..b32ca84 100644
--- a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
@@ -51,7 +51,11 @@ public class Ec2MultiRegionSnitch extends Ec2Snitch
         localPrivateAddress = awsApiCall(PRIVATE_IP_QUERY_URL);
         // use the Public IP to broadcast Address to other nodes.
         DatabaseDescriptor.setBroadcastAddress(localPublicAddress);
-        DatabaseDescriptor.setBroadcastRpcAddress(localPublicAddress);
+        if (DatabaseDescriptor.getBroadcastRpcAddress() == null)
+        {
+            logger.info("broadcast_rpc_address unset, broadcasting public IP as rpc_address: {}", localPublicAddress);
+            DatabaseDescriptor.setBroadcastRpcAddress(localPublicAddress);
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91f7387e/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 fa04595..3148f5e 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -697,7 +697,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             getTokenMetadata().updateHostId(localHostId, FBUtilities.getBroadcastAddress());
             appStates.put(ApplicationState.NET_VERSION, valueFactory.networkVersion());
             appStates.put(ApplicationState.HOST_ID, valueFactory.hostId(localHostId));
-            appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(DatabaseDescriptor.getBroadcastRpcAddress()));
+            appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(FBUtilities.getBroadcastRpcAddress()));
             appStates.put(ApplicationState.RELEASE_VERSION, valueFactory.releaseVersion());
             logger.info("Starting up server gossip");
             Gossiper.instance.register(this);
@@ -1282,7 +1282,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     public String getRpcaddress(InetAddress endpoint)
     {
         if (endpoint.equals(FBUtilities.getBroadcastAddress()))
-            return DatabaseDescriptor.getBroadcastRpcAddress().getHostAddress();
+            return FBUtilities.getBroadcastRpcAddress().getHostAddress();
         else if (Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS) == null)
             return endpoint.getHostAddress();
         else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91f7387e/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 5c0d9d2..d1047f9 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -486,7 +486,7 @@ public class Server implements CassandraDaemon.Server
             // 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(DatabaseDescriptor.getBroadcastRpcAddress()))
+                event.nodeAddress().equals(FBUtilities.getBroadcastRpcAddress()))
                 return;
 
             send(event);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91f7387e/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 9eda878..23a2c2e 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -34,6 +34,7 @@ import java.util.zip.Checksum;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.collect.AbstractIterator;
 import org.apache.commons.lang3.StringUtils;
@@ -74,6 +75,7 @@ public class FBUtilities
 
     private static volatile InetAddress localInetAddress;
     private static volatile InetAddress broadcastInetAddress;
+    private static volatile InetAddress broadcastRpcAddress;
 
     public static int getAvailableProcessors()
     {
@@ -147,6 +149,16 @@ public class FBUtilities
         return broadcastInetAddress;
     }
 
+
+    public static InetAddress getBroadcastRpcAddress()
+    {
+        if (broadcastRpcAddress == null)
+            broadcastRpcAddress = DatabaseDescriptor.getBroadcastRpcAddress() == null
+                                   ? DatabaseDescriptor.getRpcAddress()
+                                   : DatabaseDescriptor.getBroadcastRpcAddress();
+        return broadcastRpcAddress;
+    }
+
     public static Collection<InetAddress> getAllLocalAddresses()
     {
         Set<InetAddress> localAddresses = new HashSet<InetAddress>();
@@ -824,4 +836,12 @@ public class FBUtilities
             throw new RuntimeException(e);
         }
     }
+
+    @VisibleForTesting
+    protected static void reset()
+    {
+        localInetAddress = null;
+        broadcastInetAddress = null;
+        broadcastRpcAddress = null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91f7387e/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 c82bcc9..5b86252 100644
--- a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
+++ b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.utils;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.nio.charset.StandardCharsets;
@@ -30,6 +31,9 @@ import org.junit.Test;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
 import static org.junit.Assert.assertEquals;
 
 public class FBUtilitiesTest
@@ -95,4 +99,32 @@ public class FBUtilitiesTest
         ByteBuffer bytes = ByteBuffer.wrap(new byte[]{(byte)0xff, (byte)0xfe});
         ByteBufferUtil.string(bytes, StandardCharsets.UTF_8);
     }
+
+    @Test
+    public void testGetBroadcastRpcAddress() throws Exception
+    {
+        //When both rpc_address and broadcast_rpc_address are null, it should return the local address (from DD.applyAddressConfig)
+        FBUtilities.reset();
+        Config testConfig = DatabaseDescriptor.loadConfig();
+        testConfig.rpc_address = null;
+        testConfig.broadcast_rpc_address = null;
+        DatabaseDescriptor.applyAddressConfig(testConfig);
+        assertEquals(FBUtilities.getLocalAddress(), FBUtilities.getBroadcastRpcAddress());
+
+        //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());
+
+        //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());
+
+        FBUtilities.reset();
+    }
 }


[2/3] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ty...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/cassandra-3.9
Commit: 5fdb90e12b04fa4209993f609206e6920138d851
Parents: 5cef78a 91f7387
Author: Tyler Hobbs <ty...@gmail.com>
Authored: Fri Aug 12 18:29:36 2016 -0500
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Fri Aug 12 18:29:36 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  4 +--
 .../cassandra/config/DatabaseDescriptor.java    |  6 ++--
 .../cassandra/locator/Ec2MultiRegionSnitch.java |  6 +++-
 .../cassandra/service/StorageService.java       |  4 +--
 .../org/apache/cassandra/transport/Server.java  |  2 +-
 .../org/apache/cassandra/utils/FBUtilities.java | 20 ++++++++++++
 .../apache/cassandra/utils/FBUtilitiesTest.java | 32 ++++++++++++++++++++
 8 files changed, 67 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fdb90e1/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index cccb62d,394598a..7ba8370
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,40 -1,5 +1,41 @@@
 -2.2.8
 +3.0.9
 + * Fix clean interval not sent to commit log for empty memtable flush (CASSANDRA-12436)
 + * Fix potential resource leak in RMIServerSocketFactoryImpl (CASSANDRA-12331)
 + * Backport CASSANDRA-12002 (CASSANDRA-12177)
 + * Make sure compaction stats are updated when compaction is interrupted (CASSANDRA-12100)
 + * Fix potential bad messaging service message for paged range reads
 +   within mixed-version 3.x clusters (CASSANDRA-12249)
 + * Change commitlog and sstables to track dirty and clean intervals (CASSANDRA-11828)
 + * NullPointerException during compaction on table with static columns (CASSANDRA-12336)
 + * Fixed ConcurrentModificationException when reading metrics in GraphiteReporter (CASSANDRA-11823)
 + * Fix upgrade of super columns on thrift (CASSANDRA-12335)
 + * Fixed flacky BlacklistingCompactionsTest, switched to fixed size types and increased corruption size (CASSANDRA-12359)
 + * Rerun ReplicationAwareTokenAllocatorTest on failure to avoid flakiness (CASSANDRA-12277)
 + * Exception when computing read-repair for range tombstones (CASSANDRA-12263)
 + * Lost counter writes in compact table and static columns (CASSANDRA-12219)
 + * AssertionError with MVs on updating a row that isn't indexed due to a null value (CASSANDRA-12247)
 + * Disable RR and speculative retry with EACH_QUORUM reads (CASSANDRA-11980)
 + * Add option to override compaction space check (CASSANDRA-12180)
 + * Faster startup by only scanning each directory for temporary files once (CASSANDRA-12114)
 + * Respond with v1/v2 protocol header when responding to driver that attempts
 +   to connect with too low of a protocol version (CASSANDRA-11464)
 + * NullPointerExpception when reading/compacting table (CASSANDRA-11988)
 + * Fix problem with undeleteable rows on upgrade to new sstable format (CASSANDRA-12144)
 + * Fix paging logic for deleted partitions with static columns (CASSANDRA-12107)
 + * Wait until the message is being send to decide which serializer must be used (CASSANDRA-11393)
 + * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
 + * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
 + * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
 + * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
 + * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
 + * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
 + * Fix column ordering of results with static columns for Thrift requests in
 +   a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of
 +   those static columns in query results (CASSANDRA-12123)
 + * Avoid digest mismatch with empty but static rows (CASSANDRA-12090)
 + * Fix EOF exception when altering column type (CASSANDRA-11820)
 +Merged from 2.2:
+  * Only set broadcast_rpc_address on Ec2MultiRegionSnitch if it's not set (CASSANDRA-11357)
   * Update StorageProxy range metrics for timeouts, failures and unavailables (CASSANDRA-9507)
   * Add Sigar to classes included in clientutil.jar (CASSANDRA-11635)
   * Add decay to histograms and timers used for metrics (CASSANDRA-11752)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fdb90e1/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 56dd6ea,f0712eb..6b8dfd3
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,55 -13,7 +13,55 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -2.2.7
 +3.0.8
 +=====
 +
 +Upgrading
 +---------
-    - Nothing specific to this release, but please see previous versions upgrading section,
-      especially if you are upgrading from 2.2.
++   - Ec2MultiRegionSnitch will no longer automatically set broadcast_rpc_address
++     to the public instance IP if this property is defined on cassandra.yaml.
 +
 +3.0.7
 +=====
 +
 +Upgrading
 +---------
 +   - A maximum size for SSTables values has been introduced, to prevent out of memory
 +     exceptions when reading corrupt SSTables. This maximum size can be set via
 +     max_value_size_in_mb in cassandra.yaml. The default is 256MB, which matches the default
 +     value of native_transport_max_frame_size_in_mb. SSTables will be considered corrupt if
 +     they contain values whose size exceeds this limit. See CASSANDRA-9530 for more details.
 +
 +Deprecation
 +-----------
 +   - DateTieredCompactionStrategy has been deprecated - new tables should use
 +     TimeWindowCompactionStrategy. Note that migrating an existing DTCS-table to TWCS might
 +     cause increased compaction load for a while after the migration so make sure you run
 +     tests before migrating. Read CASSANDRA-9666 for background on this.
 +
 +New features
 +------------
 +   - TimeWindowCompactionStrategy has been added. This has proven to be a better approach
 +     to time series compaction and new tables should use this instead of DTCS. See
 +     CASSANDRA-9666 for details.
 +
 +3.0.6
 +=====
 +
 +New features
 +------------
 +   - JSON timestamps are now in UTC and contain the timezone information, see
 +     CASSANDRA-11137 for more details.
 +
 +3.0.5
 +=====
 +
 +Upgrading
 +---------
 +   - Nothing specific to this release, but please see previous versions upgrading section,
 +     especially if you are upgrading from 2.2.
 +
 +3.0.4
  =====
  
  New features

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fdb90e1/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fdb90e1/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fdb90e1/src/java/org/apache/cassandra/transport/Server.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fdb90e1/src/java/org/apache/cassandra/utils/FBUtilities.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/FBUtilities.java
index d996c91,23a2c2e..ca2775f
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@@ -32,7 -34,9 +32,8 @@@ import java.util.zip.Checksum
  import javax.annotation.Nonnull;
  import javax.annotation.Nullable;
  
+ import com.google.common.annotations.VisibleForTesting;
  import com.google.common.base.Joiner;
 -import com.google.common.collect.AbstractIterator;
  import org.apache.commons.lang3.StringUtils;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fdb90e1/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
index 90c5f05,5b86252..acd68eb
--- a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
+++ b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
@@@ -18,7 -18,10 +18,8 @@@
  
  package org.apache.cassandra.utils;
  
 -import static org.junit.Assert.fail;
 -
  import java.io.IOException;
+ import java.net.InetAddress;
  import java.nio.ByteBuffer;
  import java.nio.charset.CharacterCodingException;
  import java.nio.charset.StandardCharsets;
@@@ -29,8 -31,10 +30,11 @@@ import org.junit.Test
  import java.util.Map;
  import java.util.TreeMap;
  
+ import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
+ 
  import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.fail;
  
  public class FBUtilitiesTest
  {


[3/3] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by ty...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.9
Commit: d597910b2e5438962eefb5217887e844e011063a
Parents: d052c90 5fdb90e
Author: Tyler Hobbs <ty...@gmail.com>
Authored: Fri Aug 12 18:34:44 2016 -0500
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Fri Aug 12 18:34:44 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  2 ++
 .../cassandra/config/DatabaseDescriptor.java    |  6 ++--
 .../cassandra/locator/Ec2MultiRegionSnitch.java |  6 +++-
 .../cassandra/service/StorageService.java       |  4 +--
 .../org/apache/cassandra/transport/Server.java  |  2 +-
 .../org/apache/cassandra/utils/FBUtilities.java | 19 ++++++++++++
 .../apache/cassandra/utils/FBUtilitiesTest.java | 32 ++++++++++++++++++++
 8 files changed, 66 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d597910b/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d597910b/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index d8d84f5,6b8dfd3..249077d
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -45,12 -26,11 +45,14 @@@ New feature
  
  Upgrading
  ---------
 -   - A maximum size for SSTables values has been introduced, to prevent out of memory
 -     exceptions when reading corrupt SSTables. This maximum size can be set via
 -     max_value_size_in_mb in cassandra.yaml. The default is 256MB, which matches the default
 -     value of native_transport_max_frame_size_in_mb. SSTables will be considered corrupt if
 -     they contain values whose size exceeds this limit. See CASSANDRA-9530 for more details.
++    - Ec2MultiRegionSnitch will no longer automatically set broadcast_rpc_address
++      to the public instance IP if this property is defined on cassandra.yaml.
 +    - The name "json" and "distinct" are not valid anymore a user-defined function
 +      names (they are still valid as column name however). In the unlikely case where
 +      you had defined functions with such names, you will need to recreate
 +      those under a different name, change your code to use the new names and
 +      drop the old versions, and this _before_ upgrade (see CASSANDRA-10783 for more
 +      details).
  
  Deprecation
  -----------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d597910b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d597910b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 7500593,0c91784..bc69d93
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -785,14 -740,9 +785,14 @@@ public class StorageService extends Not
              getTokenMetadata().updateHostId(localHostId, FBUtilities.getBroadcastAddress());
              appStates.put(ApplicationState.NET_VERSION, valueFactory.networkVersion());
              appStates.put(ApplicationState.HOST_ID, valueFactory.hostId(localHostId));
-             appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(DatabaseDescriptor.getBroadcastRpcAddress()));
+             appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(FBUtilities.getBroadcastRpcAddress()));
              appStates.put(ApplicationState.RELEASE_VERSION, valueFactory.releaseVersion());
 +
 +            // load the persisted ring state. This used to be done earlier in the init process,
 +            // but now we always perform a shadow round when preparing to join and we have to
 +            // clear endpoint states after doing that.
 +            loadRingState();
 +
              logger.info("Starting up server gossip");
              Gossiper.instance.register(this);
              Gossiper.instance.start(SystemKeyspace.incrementAndGetGeneration(), appStates); // needed for node-ring gathering.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d597910b/src/java/org/apache/cassandra/transport/Server.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d597910b/src/java/org/apache/cassandra/utils/FBUtilities.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/FBUtilities.java
index af2cb1b,ca2775f..c76d8e1
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@@ -892,21 -872,12 +903,29 @@@ public class FBUtilitie
              throw new RuntimeException(e);
          }
      }
 +	
 +	public static void sleepQuietly(long millis)
 +    {
 +        try
 +        {
 +            Thread.sleep(millis);
 +        }
 +        catch (InterruptedException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    public static long align(long val, int boundary)
 +    {
 +        return (val + boundary) & ~(boundary - 1);
 +    }
+ 
+     @VisibleForTesting
+     protected static void reset()
+     {
+         localInetAddress = null;
+         broadcastInetAddress = null;
+         broadcastRpcAddress = null;
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d597910b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
index 3c1ea74,acd68eb..b8d2633
--- a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
+++ b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
@@@ -22,18 -23,16 +23,21 @@@ import java.net.InetAddress
  import java.nio.ByteBuffer;
  import java.nio.charset.CharacterCodingException;
  import java.nio.charset.StandardCharsets;
 +import java.util.Map;
 +import java.util.Optional;
 +import java.util.TreeMap;
  
  import com.google.common.primitives.Ints;
 +
 +import org.junit.Assert;
  import org.junit.Test;
  
 -import java.util.Map;
 -import java.util.TreeMap;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.dht.*;
  
+ import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
+ 
  import static org.junit.Assert.assertEquals;
  import static org.junit.Assert.fail;
  
@@@ -101,50 -100,31 +105,78 @@@ public class FBUtilitiesTes
          ByteBufferUtil.string(bytes, StandardCharsets.UTF_8);
      }
  
 +    private static void assertPartitioner(String name, Class expected)
 +    {
 +        Assert.assertTrue(String.format("%s != %s", name, expected.toString()),
 +                          expected.isInstance(FBUtilities.newPartitioner(name)));
 +    }
 +
 +    /**
 +     * Check that given a name, the correct partitioner instance is created.
 +     *
 +     * If the assertions in this test start failing, it likely means the sstabledump/sstablemetadata tools will
 +     * also fail to read existing sstables.
 +     */
 +    @Test
 +    public void testNewPartitionerNoArgConstructors()
 +    {
 +        assertPartitioner("ByteOrderedPartitioner", ByteOrderedPartitioner.class);
 +        assertPartitioner("LengthPartitioner", LengthPartitioner.class);
 +        assertPartitioner("Murmur3Partitioner", Murmur3Partitioner.class);
 +        assertPartitioner("OrderPreservingPartitioner", OrderPreservingPartitioner.class);
 +        assertPartitioner("RandomPartitioner", RandomPartitioner.class);
 +        assertPartitioner("org.apache.cassandra.dht.ByteOrderedPartitioner", ByteOrderedPartitioner.class);
 +        assertPartitioner("org.apache.cassandra.dht.LengthPartitioner", LengthPartitioner.class);
 +        assertPartitioner("org.apache.cassandra.dht.Murmur3Partitioner", Murmur3Partitioner.class);
 +        assertPartitioner("org.apache.cassandra.dht.OrderPreservingPartitioner", OrderPreservingPartitioner.class);
 +        assertPartitioner("org.apache.cassandra.dht.RandomPartitioner", RandomPartitioner.class);
 +    }
 +
 +    /**
 +     * Check that we can instantiate local partitioner correctly and that we can pass the correct type
 +     * to it as a constructor argument.
 +     *
 +     * If the assertions in this test start failing, it likely means the sstabledump/sstablemetadata tools will
 +     * also fail to read existing sstables.
 +     */
 +    @Test
 +    public void testNewPartitionerLocalPartitioner()
 +    {
 +        for (String name : new String[] {"LocalPartitioner", "org.apache.cassandra.dht.LocalPartitioner"})
 +            for (AbstractType<?> type : new AbstractType<?>[] {UUIDType.instance, ListType.getInstance(Int32Type.instance, true)})
 +            {
 +                IPartitioner partitioner = FBUtilities.newPartitioner(name, Optional.of(type));
 +                Assert.assertTrue(String.format("%s != LocalPartitioner", partitioner.toString()),
 +                                  LocalPartitioner.class.isInstance(partitioner));
 +                Assert.assertEquals(partitioner.partitionOrdering(), type);
 +            }
 +    }
++
+     @Test
+     public void testGetBroadcastRpcAddress() throws Exception
+     {
+         //When both rpc_address and broadcast_rpc_address are null, it should return the local address (from DD.applyAddressConfig)
+         FBUtilities.reset();
+         Config testConfig = DatabaseDescriptor.loadConfig();
+         testConfig.rpc_address = null;
+         testConfig.broadcast_rpc_address = null;
+         DatabaseDescriptor.applyAddressConfig(testConfig);
+         assertEquals(FBUtilities.getLocalAddress(), FBUtilities.getBroadcastRpcAddress());
+ 
+         //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());
+ 
+         //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());
+ 
+         FBUtilities.reset();
+     }
  }