You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jj...@apache.org on 2016/09/30 03:08:10 UTC

[1/4] cassandra git commit: Revert "Reject invalid DC names as option while creating or altering NetworkTopologyStrategy"

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 413e48e65 -> cd8a98a2d
  refs/heads/cassandra-3.X e3b34dc85 -> 02404115e
  refs/heads/trunk 87825f820 -> a052dbfe6


Revert "Reject invalid DC names as option while creating or altering NetworkTopologyStrategy"

This reverts commit f2c5ad743933498e60e7eef55e8daaa6ce338a03.


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

Branch: refs/heads/cassandra-3.0
Commit: cd8a98a2dd2d7e4993cd1aa3c282602b48be73c8
Parents: 413e48e
Author: Jeff Jirsa <je...@crowdstrike.com>
Authored: Thu Sep 29 19:50:04 2016 -0700
Committer: Jeff Jirsa <je...@crowdstrike.com>
Committed: Thu Sep 29 19:50:04 2016 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 -
 NEWS.txt                                        | 11 ----
 .../locator/AbstractReplicationStrategy.java    |  2 +-
 .../locator/NetworkTopologyStrategy.java        | 39 +------------
 .../org/apache/cassandra/cql3/CQLTester.java    | 11 ----
 .../validation/entities/SecondaryIndexTest.java | 10 ++++
 .../cql3/validation/operations/AlterTest.java   | 47 +---------------
 .../cql3/validation/operations/CreateTest.java  | 59 --------------------
 .../apache/cassandra/dht/BootStrapperTest.java  | 10 +---
 .../org/apache/cassandra/service/MoveTest.java  |  9 +--
 10 files changed, 18 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9076e7a..d12a8f8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,7 +7,6 @@
  * Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer (CASSANDRA-12516)
  * Backport CASSANDRA-10756 (race condition in NativeTransportService shutdown) (CASSANDRA-12472)
  * If CF has no clustering columns, any row cache is full partition cache (CASSANDRA-12499)
- * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 Merged from 2.2:
  * Fix merkle tree depth calculation (CASSANDRA-12580)
  * Make Collections deserialization more robust (CASSANDRA-12618)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index b97a420..0bd3920 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -13,17 +13,6 @@ restore snapshots created with the previous major version using the
 'sstableloader' tool. You can upgrade the file format of your snapshots
 using the provided 'sstableupgrade' tool.
 
-3.0.10
-=====
-
-Upgrading
----------
-   - To protect against accidental data loss, cassandra no longer allows 
-     users to set arbitrary datacenter names for NetworkTopologyStrategy. 
-     Cassandra will allow users to continue using existing keyspaces
-     with invalid datacenter names, but will validat DC names on CREATE and
-     ALTER
-
 3.0.9
 =====
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/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 d72c0c2..c90c6a1 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -319,7 +319,7 @@ public abstract class AbstractReplicationStrategy
         }
     }
 
-    protected void validateExpectedOptions() throws ConfigurationException
+    private void validateExpectedOptions() throws ConfigurationException
     {
         Collection expectedOptions = recognizedOptions();
         if (expectedOptions == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/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 78f5b06..7c8d95e 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -24,11 +24,9 @@ import java.util.Map.Entry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.TokenMetadata.Topology;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
 import com.google.common.collect.Multimap;
@@ -195,43 +193,10 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         }
     }
 
-    /*
-     * (non-javadoc) Method to generate list of valid data center names to be used to validate the replication parameters during CREATE / ALTER keyspace operations.
-     * All peers of current node are fetched from {@link TokenMetadata} and then a set is build by fetching DC name of each peer.
-     * @return a set of valid DC names
-     */
-    private static Set<String> buildValidDataCentersSet()
-    {
-        final Set<String> validDataCenters = new HashSet<>();
-        final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-
-        // Add data center of localhost.
-        validDataCenters.add(snitch.getDatacenter(FBUtilities.getBroadcastAddress()));
-        // Fetch and add DCs of all peers.
-        for (final InetAddress peer : StorageService.instance.getTokenMetadata().getAllEndpoints())
-        {
-            validDataCenters.add(snitch.getDatacenter(peer));
-        }
-
-        return validDataCenters;
-    }
-
     public Collection<String> recognizedOptions()
     {
-        // only valid options are valid DC names.
-        return buildValidDataCentersSet();
-    }
-
-    protected void validateExpectedOptions() throws ConfigurationException
-    {
-        // Do not accept query with no data centers specified.
-        if (this.configOptions.isEmpty())
-        {
-            throw new ConfigurationException("Configuration for at least one datacenter must be present");
-        }
-
-        // Validate the data center names
-        super.validateExpectedOptions();
+        // We explicitely allow all options
+        return null;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/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 69a0b79..7f5eb02 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -56,8 +56,6 @@ import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.locator.AbstractEndpointSnitch;
-import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
@@ -84,8 +82,6 @@ public abstract class CQLTester
     protected static final long ROW_CACHE_SIZE_IN_MB = Integer.valueOf(System.getProperty("cassandra.test.row_cache_size_in_mb", "0"));
     private static final AtomicInteger seqNumber = new AtomicInteger();
     protected static final ByteBuffer TOO_BIG = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT + 1024);
-    public static final String DATA_CENTER = "datacenter1";
-    public static final String RACK1 = "rack1";
 
     private static org.apache.cassandra.transport.Server server;
     protected static final int nativePort;
@@ -131,13 +127,6 @@ public abstract class CQLTester
         {
             throw new RuntimeException(e);
         }
-        // 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; }
-        });
     }
 
     public static ResultMessage lastSchemaChangeResult;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 2b31481..0cf13bd 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -239,6 +239,16 @@ public class SecondaryIndexTest extends CQLTester
     }
 
     /**
+     * Check one can use arbitrary name for datacenter when creating keyspace (#4278),
+     * migrated from cql_tests.py:TestCQL.keyspace_creation_options_test()
+     */
+    @Test
+    public void testDataCenterName() throws Throwable
+    {
+       execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east' : 1, 'us-west' : 1 };");
+    }
+
+    /**
      * Migrated from cql_tests.py:TestCQL.indexes_composite_test()
      */
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index 48108cd..bcd6587 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -212,13 +212,13 @@ public class AlterTest extends CQLTester
                    row(ks1, true),
                    row(ks2, false));
 
-        schemaChange("ALTER KEYSPACE " + ks1 + " WITH replication = { 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 1 } AND durable_writes=False");
+        schemaChange("ALTER KEYSPACE " + ks1 + " WITH replication = { 'class' : 'NetworkTopologyStrategy', 'dc1' : 1 } AND durable_writes=False");
         schemaChange("ALTER KEYSPACE " + ks2 + " WITH durable_writes=true");
 
         assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
                    row(KEYSPACE, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")),
                    row(KEYSPACE_PER_TEST, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")),
-                   row(ks1, false, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER , "1")),
+                   row(ks1, false, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", "dc1", "1")),
                    row(ks2, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")));
 
         execute("USE " + ks1);
@@ -233,49 +233,6 @@ public class AlterTest extends CQLTester
     }
 
     /**
-     * Test {@link ConfigurationException} thrown on alter keyspace to no DC option in replication configuration.
-     */
-    @Test
-    public void testAlterKeyspaceWithNoOptionThrowsConfigurationException() throws Throwable
-    {
-        // Create keyspaces
-        execute("CREATE KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
-        execute("CREATE KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 3 }");
-
-        // Try to alter the created keyspace without any option
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy' }");
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy' }");
-
-        // Make sure that the alter works as expected
-        execute("ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-        execute("ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
-
-        // clean up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-        execute("DROP KEYSPACE IF EXISTS testXYZ");
-    }
-
-    /**
-     * Test {@link ConfigurationException} thrown when altering a keyspace to invalid DC option in replication configuration.
-     */
-    @Test
-    public void testAlterKeyspaceWithNTSOnlyAcceptsConfiguredDataCenterNames() throws Throwable
-    {
-        // Create a keyspace with expected DC name.
-        execute("CREATE KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-
-        // try modifying the keyspace
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication = { 'class' : 'NetworkTopologyStrategy', 'INVALID_DC' : 2 }");
-        execute("ALTER KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
-
-        // Mix valid and invalid, should throw an exception
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 , 'INVALID_DC': 1}");
-
-        // clean-up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-    }
-
-    /**
      * Test for bug of 5232,
      * migrated from cql_tests.py:TestCQL.alter_bug_test()
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/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 0781169..33a41d8 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.util.Collection;
 import java.util.Collections;
 import java.util.UUID;
@@ -26,15 +25,12 @@ import org.junit.Test;
 
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.locator.AbstractEndpointSnitch;
-import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.schema.SchemaKeyspace;
 import org.apache.cassandra.triggers.ITrigger;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -349,33 +345,6 @@ public class CreateTest extends CQLTester
     }
 
     /**
-     *  Test {@link ConfigurationException} is thrown on create keyspace with invalid DC option in replication configuration .
-     */
-    @Test
-    public void testCreateKeyspaceWithNTSOnlyAcceptsConfiguredDataCenterNames() throws Throwable
-    {
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testABC WITH replication = { 'class' : 'NetworkTopologyStrategy', 'INVALID_DC' : 2 }");
-        execute("CREATE KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-
-        // Mix valid and invalid, should throw an exception
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 , 'INVALID_DC': 1}");
-
-        // clean-up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-        execute("DROP KEYSPACE IF EXISTS testXYZ");
-    }
-
-    /**
-     * Test {@link ConfigurationException} is thrown on create keyspace without any options.
-     */
-    @Test
-    public void testConfigurationExceptionThrownWhenCreateKeyspaceWithNoOptions() throws Throwable
-    {
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ with replication = { 'class': 'NetworkTopologyStrategy' }");
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy' }");
-    }
-
-    /**
      * Test create and drop table
      * migrated from cql_tests.py:TestCQL.table_test()
      */
@@ -526,34 +495,6 @@ public class CreateTest extends CQLTester
     }
 
     @Test
-    // tests CASSANDRA-4278
-    public void testHyphenDatacenters() throws Throwable
-    {
-        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-
-        // 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 "us-east-1"; }
-
-            @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2) { return 0; }
-        });
-
-        execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east-1' : 1 };");
-
-        // Restore the previous EndpointSnitch
-        DatabaseDescriptor.setEndpointSnitch(snitch);
-
-        // Clean up
-        execute("DROP KEYSPACE IF EXISTS Foo");
-    }
-
-    @Test
     // tests CASSANDRA-9565
     public void testDoubleWith() throws Throwable
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/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 cd34aea..8974791 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -27,7 +27,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 
 import com.google.common.collect.Lists;
 
@@ -60,7 +59,7 @@ public class BootStrapperTest
     static IPartitioner oldPartitioner;
 
     @BeforeClass
-    public static void setup() throws Exception
+    public static void setup() throws ConfigurationException
     {
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
         SchemaLoader.startGossiper();
@@ -178,13 +177,6 @@ public class BootStrapperTest
             int vn = 16;
             String ks = "BootStrapperTestNTSKeyspace" + rackCount + replicas;
             String dc = "1";
-
-            // 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"));
-
             SchemaLoader.createKeyspace(ks, KeyspaceParams.nts(dc, replicas, "15", 15), SchemaLoader.standardCFMD(ks, "Standard1"));
             TokenMetadata tm = new TokenMetadata();
             tm.clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cd8a98a2/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 6c07a47..53365aa 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -82,7 +82,7 @@ public class MoveTest
      * So instead of extending SchemaLoader, we call it's method below.
      */
     @BeforeClass
-    public static void setup() throws Exception
+    public static void setup() throws ConfigurationException
     {
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
         SchemaLoader.loadSchema();
@@ -105,7 +105,7 @@ public class MoveTest
         StorageService.instance.getTokenMetadata().clearUnsafe();
     }
 
-    private static void addNetworkTopologyKeyspace(String keyspaceName, Integer... replicas) throws Exception
+    private static void addNetworkTopologyKeyspace(String keyspaceName, Integer... replicas) throws ConfigurationException
     {
 
         DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch()
@@ -139,11 +139,6 @@ public class MoveTest
             }
         });
 
-        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"));
-
         KeyspaceMetadata keyspace =  KeyspaceMetadata.create(keyspaceName,
                                                              KeyspaceParams.nts(configOptions(replicas)),
                                                              Tables.of(CFMetaData.Builder.create(keyspaceName, "CF1")


[4/4] cassandra git commit: Merge branch 'cassandra-3.X' into trunk

Posted by jj...@apache.org.
Merge branch 'cassandra-3.X' into trunk


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

Branch: refs/heads/trunk
Commit: a052dbfe6fe6d513f3898216ee42325e6ca5e8a4
Parents: 87825f8 0240411
Author: Jeff Jirsa <je...@crowdstrike.com>
Authored: Thu Sep 29 20:02:31 2016 -0700
Committer: Jeff Jirsa <je...@crowdstrike.com>
Committed: Thu Sep 29 20:05:38 2016 -0700

----------------------------------------------------------------------
 CHANGES.txt | 3 +--
 NEWS.txt    | 8 ++++----
 2 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a052dbfe/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 727ad64,ca95411..4472411
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,6 -1,3 +1,6 @@@
 +4.0
- 
++ * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 +
  3.10
   * cdc column addition still breaks schema migration tasks (CASSANDRA-12697)
   * Upgrade metrics-reporter dependencies (CASSANDRA-12089)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a052dbfe/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index e1e952b,ad0f2be..9db425a
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,16 -13,6 +13,19 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 +4.0
 +===
 +
 +New features
 +------------
 +
 +Upgrading
 +---------
- 
++    - Cassandra will no longer allow invalid keyspace replication options, such as invalid datacenter names for
++      NetworkTopologyStrategy. Operators MUST add new nodes to a datacenter before they can set set ALTER or 
++      CREATE keyspace replication policies using that datacenter. Existing keyspaces will continue to operate, 
++      but CREATE and ALTER will validate that all datacenters specified exist in the cluster. 
 +
  3.10
  ====
  


[3/4] cassandra git commit: Revert "Merge branch 'cassandra-3.0' into trunk"

Posted by jj...@apache.org.
Revert "Merge branch 'cassandra-3.0' into trunk"

This reverts commit d45f323eb972c6fec146e5cfa84fdc47eb8aa5eb, reversing
changes made to b80ef9b2580c123da90879b4456606ef5b01b6f2.


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

Branch: refs/heads/trunk
Commit: 02404115e0db1762473556f985c8e5388db92da6
Parents: e3b34dc
Author: Jeff Jirsa <je...@crowdstrike.com>
Authored: Thu Sep 29 20:00:41 2016 -0700
Committer: Jeff Jirsa <je...@crowdstrike.com>
Committed: Thu Sep 29 20:00:41 2016 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 -
 NEWS.txt                                        |  3 -
 .../locator/AbstractReplicationStrategy.java    |  2 +-
 .../locator/NetworkTopologyStrategy.java        | 41 --------------
 .../org/apache/cassandra/cql3/CQLTester.java    | 11 ----
 .../validation/entities/SecondaryIndexTest.java | 10 ++++
 .../cql3/validation/operations/AlterTest.java   | 47 +---------------
 .../cql3/validation/operations/CreateTest.java  | 59 --------------------
 .../apache/cassandra/dht/BootStrapperTest.java  |  8 ---
 .../org/apache/cassandra/service/MoveTest.java  |  9 +--
 10 files changed, 15 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ba08745..ca95411 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -92,7 +92,6 @@ Merged from 3.0:
  * Calculate last compacted key on startup (CASSANDRA-6216)
  * Add schema to snapshot manifest, add USING TIMESTAMP clause to ALTER TABLE statements (CASSANDRA-7190)
  * If CF has no clustering columns, any row cache is full partition cache (CASSANDRA-12499)
- * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 Merged from 2.2:
  * Fix merkle tree depth calculation (CASSANDRA-12580)
  * Make Collections deserialization more robust (CASSANDRA-12618)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 9ab7c26..ad0f2be 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -98,9 +98,6 @@ Upgrading
     - Application layer keep-alives were added to the streaming protocol to prevent idle incoming connections from
       timing out and failing the stream session (CASSANDRA-11839). This effectively deprecates the streaming_socket_timeout_in_ms
       property in favor of streaming_keep_alive_period_in_secs. See cassandra.yaml for more details about this property.
-    - Cassandra will no longer allow invalid keyspace replication options, such as invalid datacenter names for
-      NetworkTopologyStrategy. Existing keyspaces will continue to operate, but CREATE and ALTER will validate that
-      all datacenters specified exist in the cluster. 
 
 3.8
 ===

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 4ba1da8..038ac9f 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -320,7 +320,7 @@ public abstract class AbstractReplicationStrategy
         }
     }
 
-    protected void validateExpectedOptions() throws ConfigurationException
+    private void validateExpectedOptions() throws ConfigurationException
     {
         Collection expectedOptions = recognizedOptions();
         if (expectedOptions == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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..756b689 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -24,11 +24,9 @@ import java.util.Map.Entry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.TokenMetadata.Topology;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
@@ -216,45 +214,6 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         return datacenters.keySet();
     }
 
-    /*
-     * (non-javadoc) Method to generate list of valid data center names to be used to validate the replication parameters during CREATE / ALTER keyspace operations.
-     * All peers of current node are fetched from {@link TokenMetadata} and then a set is build by fetching DC name of each peer.
-     * @return a set of valid DC names
-     */
-    private static Set<String> buildValidDataCentersSet()
-    {
-        final Set<String> validDataCenters = new HashSet<>();
-        final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-
-        // Add data center of localhost.
-        validDataCenters.add(snitch.getDatacenter(FBUtilities.getBroadcastAddress()));
-        // Fetch and add DCs of all peers.
-        for (final InetAddress peer : StorageService.instance.getTokenMetadata().getAllEndpoints())
-        {
-            validDataCenters.add(snitch.getDatacenter(peer));
-        }
-
-        return validDataCenters;
-    }
-
-    public Collection<String> recognizedOptions()
-    {
-        // only valid options are valid DC names.
-        return buildValidDataCentersSet();
-    }
-
-    protected void validateExpectedOptions() throws ConfigurationException
-    {
-        // Do not accept query with no data centers specified.
-        if (this.configOptions.isEmpty())
-        {
-            throw new ConfigurationException("Configuration for at least one datacenter must be present");
-        }
-
-        // Validate the data center names
-        super.validateExpectedOptions();
-    }
-
     public void validateOptions() throws ConfigurationException
     {
         for (Entry<String, String> e : this.configOptions.entrySet())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 29d51a8..3bb753f 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -57,7 +57,6 @@ import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.locator.AbstractEndpointSnitch;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
@@ -84,8 +83,6 @@ public abstract class CQLTester
     protected static final long ROW_CACHE_SIZE_IN_MB = Integer.valueOf(System.getProperty("cassandra.test.row_cache_size_in_mb", "0"));
     private static final AtomicInteger seqNumber = new AtomicInteger();
     protected static final ByteBuffer TOO_BIG = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT + 1024);
-    public static final String DATA_CENTER = "datacenter1";
-    public static final String RACK1 = "rack1";
 
     private static org.apache.cassandra.transport.Server server;
     protected static final int nativePort;
@@ -118,14 +115,6 @@ public abstract class CQLTester
 
         nativeAddr = InetAddress.getLoopbackAddress();
 
-        // 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; }
-        });
-
         try
         {
             try (ServerSocket serverSocket = new ServerSocket(0))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 5af3ebb..5cb76c9 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -246,6 +246,16 @@ public class SecondaryIndexTest extends CQLTester
     }
 
     /**
+     * Check one can use arbitrary name for datacenter when creating keyspace (#4278),
+     * migrated from cql_tests.py:TestCQL.keyspace_creation_options_test()
+     */
+    @Test
+    public void testDataCenterName() throws Throwable
+    {
+       execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east' : 1, 'us-west' : 1 };");
+    }
+
+    /**
      * Migrated from cql_tests.py:TestCQL.indexes_composite_test()
      */
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index 672b6eb..c9be678 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -212,13 +212,13 @@ public class AlterTest extends CQLTester
                    row(ks1, true),
                    row(ks2, false));
 
-        schemaChange("ALTER KEYSPACE " + ks1 + " WITH replication = { 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 1 } AND durable_writes=False");
+        schemaChange("ALTER KEYSPACE " + ks1 + " WITH replication = { 'class' : 'NetworkTopologyStrategy', 'dc1' : 1 } AND durable_writes=False");
         schemaChange("ALTER KEYSPACE " + ks2 + " WITH durable_writes=true");
 
         assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
                    row(KEYSPACE, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")),
                    row(KEYSPACE_PER_TEST, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")),
-                   row(ks1, false, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER, "1")),
+                   row(ks1, false, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", "dc1", "1")),
                    row(ks2, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")));
 
         execute("USE " + ks1);
@@ -233,49 +233,6 @@ public class AlterTest extends CQLTester
     }
 
     /**
-     * Test {@link ConfigurationException} thrown on alter keyspace to no DC option in replication configuration.
-     */
-    @Test
-    public void testAlterKeyspaceWithNoOptionThrowsConfigurationException() throws Throwable
-    {
-        // Create keyspaces
-        execute("CREATE KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
-        execute("CREATE KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 3 }");
-
-        // Try to alter the created keyspace without any option
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy' }");
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy' }");
-
-        // Make sure that the alter works as expected
-        execute("ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-        execute("ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
-
-        // clean up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-        execute("DROP KEYSPACE IF EXISTS testXYZ");
-    }
-
-    /**
-     * Test {@link ConfigurationException} thrown when altering a keyspace to invalid DC option in replication configuration.
-     */
-    @Test
-    public void testAlterKeyspaceWithNTSOnlyAcceptsConfiguredDataCenterNames() throws Throwable
-    {
-        // Create a keyspace with expected DC name.
-        execute("CREATE KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-
-        // try modifying the keyspace
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication = { 'class' : 'NetworkTopologyStrategy', 'INVALID_DC' : 2 }");
-        execute("ALTER KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
-
-        // Mix valid and invalid, should throw an exception
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 , 'INVALID_DC': 1}");
-
-        // clean-up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-    }
-
-    /**
      * Test for bug of 5232,
      * migrated from cql_tests.py:TestCQL.alter_bug_test()
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 4957b18..da0824f 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.util.Collection;
 import java.util.Collections;
 import java.util.UUID;
@@ -25,7 +24,6 @@ import java.util.UUID;
 import org.junit.Test;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
@@ -33,8 +31,6 @@ import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.locator.AbstractEndpointSnitch;
-import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.schema.SchemaKeyspace;
 import org.apache.cassandra.triggers.ITrigger;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -349,33 +345,6 @@ public class CreateTest extends CQLTester
     }
 
     /**
-     *  Test {@link ConfigurationException} is thrown on create keyspace with invalid DC option in replication configuration .
-     */
-    @Test
-    public void testCreateKeyspaceWithNTSOnlyAcceptsConfiguredDataCenterNames() throws Throwable
-    {
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testABC WITH replication = { 'class' : 'NetworkTopologyStrategy', 'INVALID_DC' : 2 }");
-        execute("CREATE KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-
-        // Mix valid and invalid, should throw an exception
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 , 'INVALID_DC': 1}");
-
-        // clean-up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-        execute("DROP KEYSPACE IF EXISTS testXYZ");
-    }
-
-    /**
-     * Test {@link ConfigurationException} is thrown on create keyspace without any options.
-     */
-    @Test
-    public void testConfigurationExceptionThrownWhenCreateKeyspaceWithNoOptions() throws Throwable
-    {
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ with replication = { 'class': 'NetworkTopologyStrategy' }");
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy' }");
-    }
-
-    /**
      * Test create and drop table
      * migrated from cql_tests.py:TestCQL.table_test()
      */
@@ -526,34 +495,6 @@ public class CreateTest extends CQLTester
     }
 
     @Test
-    // tests CASSANDRA-4278
-    public void testHyphenDatacenters() throws Throwable
-    {
-        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-
-        // 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 "us-east-1"; }
-
-            @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2) { return 0; }
-        });
-
-        execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east-1' : 1 };");
-
-        // Restore the previous EndpointSnitch
-        DatabaseDescriptor.setEndpointSnitch(snitch);
-
-        // clean up
-        execute("DROP KEYSPACE IF EXISTS Foo");
-    }
-
-    @Test
     // tests CASSANDRA-9565
     public void testDoubleWith() throws Throwable
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 715f7dd..7662faa 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -27,7 +27,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 
 import com.google.common.collect.Lists;
 
@@ -179,13 +178,6 @@ public class BootStrapperTest
             int vn = 16;
             String ks = "BootStrapperTestNTSKeyspace" + rackCount + replicas;
             String dc = "1";
-
-            // 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"));
-
             SchemaLoader.createKeyspace(ks, KeyspaceParams.nts(dc, replicas, "15", 15), SchemaLoader.standardCFMD(ks, "Standard1"));
             TokenMetadata tm = new TokenMetadata();
             tm.clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 e8d5ccd..05757c0 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -82,7 +82,7 @@ public class MoveTest
      * So instead of extending SchemaLoader, we call it's method below.
      */
     @BeforeClass
-    public static void setup() throws Exception
+    public static void setup() throws ConfigurationException
     {
         DatabaseDescriptor.daemonInitialization();
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
@@ -106,7 +106,7 @@ public class MoveTest
         StorageService.instance.getTokenMetadata().clearUnsafe();
     }
 
-    private static void addNetworkTopologyKeyspace(String keyspaceName, Integer... replicas) throws Exception
+    private static void addNetworkTopologyKeyspace(String keyspaceName, Integer... replicas) throws ConfigurationException
     {
 
         DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch()
@@ -140,11 +140,6 @@ public class MoveTest
             }
         });
 
-        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"));
-
         KeyspaceMetadata keyspace =  KeyspaceMetadata.create(keyspaceName,
                                                              KeyspaceParams.nts(configOptions(replicas)),
                                                              Tables.of(CFMetaData.Builder.create(keyspaceName, "CF1")


[2/4] cassandra git commit: Revert "Merge branch 'cassandra-3.0' into trunk"

Posted by jj...@apache.org.
Revert "Merge branch 'cassandra-3.0' into trunk"

This reverts commit d45f323eb972c6fec146e5cfa84fdc47eb8aa5eb, reversing
changes made to b80ef9b2580c123da90879b4456606ef5b01b6f2.


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

Branch: refs/heads/cassandra-3.X
Commit: 02404115e0db1762473556f985c8e5388db92da6
Parents: e3b34dc
Author: Jeff Jirsa <je...@crowdstrike.com>
Authored: Thu Sep 29 20:00:41 2016 -0700
Committer: Jeff Jirsa <je...@crowdstrike.com>
Committed: Thu Sep 29 20:00:41 2016 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 -
 NEWS.txt                                        |  3 -
 .../locator/AbstractReplicationStrategy.java    |  2 +-
 .../locator/NetworkTopologyStrategy.java        | 41 --------------
 .../org/apache/cassandra/cql3/CQLTester.java    | 11 ----
 .../validation/entities/SecondaryIndexTest.java | 10 ++++
 .../cql3/validation/operations/AlterTest.java   | 47 +---------------
 .../cql3/validation/operations/CreateTest.java  | 59 --------------------
 .../apache/cassandra/dht/BootStrapperTest.java  |  8 ---
 .../org/apache/cassandra/service/MoveTest.java  |  9 +--
 10 files changed, 15 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ba08745..ca95411 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -92,7 +92,6 @@ Merged from 3.0:
  * Calculate last compacted key on startup (CASSANDRA-6216)
  * Add schema to snapshot manifest, add USING TIMESTAMP clause to ALTER TABLE statements (CASSANDRA-7190)
  * If CF has no clustering columns, any row cache is full partition cache (CASSANDRA-12499)
- * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 Merged from 2.2:
  * Fix merkle tree depth calculation (CASSANDRA-12580)
  * Make Collections deserialization more robust (CASSANDRA-12618)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 9ab7c26..ad0f2be 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -98,9 +98,6 @@ Upgrading
     - Application layer keep-alives were added to the streaming protocol to prevent idle incoming connections from
       timing out and failing the stream session (CASSANDRA-11839). This effectively deprecates the streaming_socket_timeout_in_ms
       property in favor of streaming_keep_alive_period_in_secs. See cassandra.yaml for more details about this property.
-    - Cassandra will no longer allow invalid keyspace replication options, such as invalid datacenter names for
-      NetworkTopologyStrategy. Existing keyspaces will continue to operate, but CREATE and ALTER will validate that
-      all datacenters specified exist in the cluster. 
 
 3.8
 ===

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 4ba1da8..038ac9f 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -320,7 +320,7 @@ public abstract class AbstractReplicationStrategy
         }
     }
 
-    protected void validateExpectedOptions() throws ConfigurationException
+    private void validateExpectedOptions() throws ConfigurationException
     {
         Collection expectedOptions = recognizedOptions();
         if (expectedOptions == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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..756b689 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -24,11 +24,9 @@ import java.util.Map.Entry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.TokenMetadata.Topology;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
@@ -216,45 +214,6 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         return datacenters.keySet();
     }
 
-    /*
-     * (non-javadoc) Method to generate list of valid data center names to be used to validate the replication parameters during CREATE / ALTER keyspace operations.
-     * All peers of current node are fetched from {@link TokenMetadata} and then a set is build by fetching DC name of each peer.
-     * @return a set of valid DC names
-     */
-    private static Set<String> buildValidDataCentersSet()
-    {
-        final Set<String> validDataCenters = new HashSet<>();
-        final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-
-        // Add data center of localhost.
-        validDataCenters.add(snitch.getDatacenter(FBUtilities.getBroadcastAddress()));
-        // Fetch and add DCs of all peers.
-        for (final InetAddress peer : StorageService.instance.getTokenMetadata().getAllEndpoints())
-        {
-            validDataCenters.add(snitch.getDatacenter(peer));
-        }
-
-        return validDataCenters;
-    }
-
-    public Collection<String> recognizedOptions()
-    {
-        // only valid options are valid DC names.
-        return buildValidDataCentersSet();
-    }
-
-    protected void validateExpectedOptions() throws ConfigurationException
-    {
-        // Do not accept query with no data centers specified.
-        if (this.configOptions.isEmpty())
-        {
-            throw new ConfigurationException("Configuration for at least one datacenter must be present");
-        }
-
-        // Validate the data center names
-        super.validateExpectedOptions();
-    }
-
     public void validateOptions() throws ConfigurationException
     {
         for (Entry<String, String> e : this.configOptions.entrySet())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 29d51a8..3bb753f 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -57,7 +57,6 @@ import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.locator.AbstractEndpointSnitch;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
@@ -84,8 +83,6 @@ public abstract class CQLTester
     protected static final long ROW_CACHE_SIZE_IN_MB = Integer.valueOf(System.getProperty("cassandra.test.row_cache_size_in_mb", "0"));
     private static final AtomicInteger seqNumber = new AtomicInteger();
     protected static final ByteBuffer TOO_BIG = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT + 1024);
-    public static final String DATA_CENTER = "datacenter1";
-    public static final String RACK1 = "rack1";
 
     private static org.apache.cassandra.transport.Server server;
     protected static final int nativePort;
@@ -118,14 +115,6 @@ public abstract class CQLTester
 
         nativeAddr = InetAddress.getLoopbackAddress();
 
-        // 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; }
-        });
-
         try
         {
             try (ServerSocket serverSocket = new ServerSocket(0))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 5af3ebb..5cb76c9 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -246,6 +246,16 @@ public class SecondaryIndexTest extends CQLTester
     }
 
     /**
+     * Check one can use arbitrary name for datacenter when creating keyspace (#4278),
+     * migrated from cql_tests.py:TestCQL.keyspace_creation_options_test()
+     */
+    @Test
+    public void testDataCenterName() throws Throwable
+    {
+       execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east' : 1, 'us-west' : 1 };");
+    }
+
+    /**
      * Migrated from cql_tests.py:TestCQL.indexes_composite_test()
      */
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index 672b6eb..c9be678 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -212,13 +212,13 @@ public class AlterTest extends CQLTester
                    row(ks1, true),
                    row(ks2, false));
 
-        schemaChange("ALTER KEYSPACE " + ks1 + " WITH replication = { 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 1 } AND durable_writes=False");
+        schemaChange("ALTER KEYSPACE " + ks1 + " WITH replication = { 'class' : 'NetworkTopologyStrategy', 'dc1' : 1 } AND durable_writes=False");
         schemaChange("ALTER KEYSPACE " + ks2 + " WITH durable_writes=true");
 
         assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
                    row(KEYSPACE, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")),
                    row(KEYSPACE_PER_TEST, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")),
-                   row(ks1, false, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER, "1")),
+                   row(ks1, false, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", "dc1", "1")),
                    row(ks2, true, map("class", "org.apache.cassandra.locator.SimpleStrategy", "replication_factor", "1")));
 
         execute("USE " + ks1);
@@ -233,49 +233,6 @@ public class AlterTest extends CQLTester
     }
 
     /**
-     * Test {@link ConfigurationException} thrown on alter keyspace to no DC option in replication configuration.
-     */
-    @Test
-    public void testAlterKeyspaceWithNoOptionThrowsConfigurationException() throws Throwable
-    {
-        // Create keyspaces
-        execute("CREATE KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
-        execute("CREATE KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 3 }");
-
-        // Try to alter the created keyspace without any option
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy' }");
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy' }");
-
-        // Make sure that the alter works as expected
-        execute("ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-        execute("ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
-
-        // clean up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-        execute("DROP KEYSPACE IF EXISTS testXYZ");
-    }
-
-    /**
-     * Test {@link ConfigurationException} thrown when altering a keyspace to invalid DC option in replication configuration.
-     */
-    @Test
-    public void testAlterKeyspaceWithNTSOnlyAcceptsConfiguredDataCenterNames() throws Throwable
-    {
-        // Create a keyspace with expected DC name.
-        execute("CREATE KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-
-        // try modifying the keyspace
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication = { 'class' : 'NetworkTopologyStrategy', 'INVALID_DC' : 2 }");
-        execute("ALTER KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
-
-        // Mix valid and invalid, should throw an exception
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 , 'INVALID_DC': 1}");
-
-        // clean-up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-    }
-
-    /**
      * Test for bug of 5232,
      * migrated from cql_tests.py:TestCQL.alter_bug_test()
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 4957b18..da0824f 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.util.Collection;
 import java.util.Collections;
 import java.util.UUID;
@@ -25,7 +24,6 @@ import java.util.UUID;
 import org.junit.Test;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
@@ -33,8 +31,6 @@ import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.locator.AbstractEndpointSnitch;
-import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.schema.SchemaKeyspace;
 import org.apache.cassandra.triggers.ITrigger;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -349,33 +345,6 @@ public class CreateTest extends CQLTester
     }
 
     /**
-     *  Test {@link ConfigurationException} is thrown on create keyspace with invalid DC option in replication configuration .
-     */
-    @Test
-    public void testCreateKeyspaceWithNTSOnlyAcceptsConfiguredDataCenterNames() throws Throwable
-    {
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testABC WITH replication = { 'class' : 'NetworkTopologyStrategy', 'INVALID_DC' : 2 }");
-        execute("CREATE KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-
-        // Mix valid and invalid, should throw an exception
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 , 'INVALID_DC': 1}");
-
-        // clean-up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-        execute("DROP KEYSPACE IF EXISTS testXYZ");
-    }
-
-    /**
-     * Test {@link ConfigurationException} is thrown on create keyspace without any options.
-     */
-    @Test
-    public void testConfigurationExceptionThrownWhenCreateKeyspaceWithNoOptions() throws Throwable
-    {
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ with replication = { 'class': 'NetworkTopologyStrategy' }");
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy' }");
-    }
-
-    /**
      * Test create and drop table
      * migrated from cql_tests.py:TestCQL.table_test()
      */
@@ -526,34 +495,6 @@ public class CreateTest extends CQLTester
     }
 
     @Test
-    // tests CASSANDRA-4278
-    public void testHyphenDatacenters() throws Throwable
-    {
-        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-
-        // 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 "us-east-1"; }
-
-            @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2) { return 0; }
-        });
-
-        execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east-1' : 1 };");
-
-        // Restore the previous EndpointSnitch
-        DatabaseDescriptor.setEndpointSnitch(snitch);
-
-        // clean up
-        execute("DROP KEYSPACE IF EXISTS Foo");
-    }
-
-    @Test
     // tests CASSANDRA-9565
     public void testDoubleWith() throws Throwable
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 715f7dd..7662faa 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -27,7 +27,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 
 import com.google.common.collect.Lists;
 
@@ -179,13 +178,6 @@ public class BootStrapperTest
             int vn = 16;
             String ks = "BootStrapperTestNTSKeyspace" + rackCount + replicas;
             String dc = "1";
-
-            // 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"));
-
             SchemaLoader.createKeyspace(ks, KeyspaceParams.nts(dc, replicas, "15", 15), SchemaLoader.standardCFMD(ks, "Standard1"));
             TokenMetadata tm = new TokenMetadata();
             tm.clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02404115/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 e8d5ccd..05757c0 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -82,7 +82,7 @@ public class MoveTest
      * So instead of extending SchemaLoader, we call it's method below.
      */
     @BeforeClass
-    public static void setup() throws Exception
+    public static void setup() throws ConfigurationException
     {
         DatabaseDescriptor.daemonInitialization();
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
@@ -106,7 +106,7 @@ public class MoveTest
         StorageService.instance.getTokenMetadata().clearUnsafe();
     }
 
-    private static void addNetworkTopologyKeyspace(String keyspaceName, Integer... replicas) throws Exception
+    private static void addNetworkTopologyKeyspace(String keyspaceName, Integer... replicas) throws ConfigurationException
     {
 
         DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch()
@@ -140,11 +140,6 @@ public class MoveTest
             }
         });
 
-        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"));
-
         KeyspaceMetadata keyspace =  KeyspaceMetadata.create(keyspaceName,
                                                              KeyspaceParams.nts(configOptions(replicas)),
                                                              Tables.of(CFMetaData.Builder.create(keyspaceName, "CF1")