You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ms...@apache.org on 2016/08/16 22:42:31 UTC

[30/50] [abbrv] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

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.8
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();
+     }
  }