Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 626ac94cd -> 711870e34


Don't allow startup if the node's rack has changed

patch by Carl Yeksigian; revieiwed by Paulo Motta for CASSANDRA-10242


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

Branch: refs/heads/cassandra-2.1
Commit: 711870e3484a7ebec5fe47b2a90bf8f8b3d1e0af
Parents: 626ac94
Author: Carl Yeksigian <c...@apache.org>
Authored: Mon Oct 12 12:47:16 2015 -0400
Committer: Aleksey Yeschenko <alek...@apache.org>
Committed: Wed Oct 14 15:35:09 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  3 +-
 NEWS.txt                                        | 13 ++++++
 conf/cassandra-rackdc.properties                |  3 ++
 conf/cassandra.yaml                             |  3 ++
 .../org/apache/cassandra/db/SystemKeyspace.java | 49 ++++++++++++++------
 5 files changed, 56 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/711870e3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index cd2a779..9a82780 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 2.1.12
-  * (cqlsh) show partial trace if incomplete after max_trace_wait 
(CASSANDRA-7645)
+ * Don't allow startup if the node's rack has changed (CASSANDRA-10242)
+ * (cqlsh) show partial trace if incomplete after max_trace_wait 
(CASSANDRA-7645)
 
 
 2.1.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/711870e3/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 4b34778..fadd541 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -13,6 +13,18 @@ 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.
 
+2.1.12
+======
+
+New features
+------------
+    - Switching racks is no longer an allowed operation on a node which has
+      data. Instead, the node will need to be wiped and bootstrapped. If
+      moving from the SimpleSnitch, make sure the rack containing all current
+      nodes is named "rack1". To override this behavior when manually wiping
+      the node and bootstrapping, use -Dcassandra.ignore_rack=true.
+
+
 2.1.11
 =====
 
@@ -21,6 +33,7 @@ Upgrading
     - Nothing specific to this release, but please see 2.1 if you are upgrading
       from a previous version.
 
+
 2.1.10
 =====
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/711870e3/conf/cassandra-rackdc.properties
----------------------------------------------------------------------
diff --git a/conf/cassandra-rackdc.properties b/conf/cassandra-rackdc.properties
index f0a0d55..f85646e 100644
--- a/conf/cassandra-rackdc.properties
+++ b/conf/cassandra-rackdc.properties
@@ -16,6 +16,9 @@
 
 # These properties are used with GossipingPropertyFileSnitch and will
 # indicate the rack and dc for this node
+#
+# When upgrading from SimpleSnitch, you will need to set your initial machines
+# to have rack=rack1
 dc=DC1
 rack=RAC1
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/711870e3/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 7e266d1..df1aa15 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -654,6 +654,9 @@ cross_node_timeout: false
 # YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS
 # ARE PLACED.
 #
+# IF THE RACK A REPLICA IS PLACED IN CHANGES AFTER THE REPLICA HAS BEEN
+# ADDED TO A RING, THE NODE MUST BE WIPED AND REBOOTSTRAPPED.
+#
 # Out of the box, Cassandra provides
 #  - SimpleSnitch:
 #    Treats Strategy order as proximity. This can improve cache

http://git-wip-us.apache.org/repos/asf/cassandra/blob/711870e3/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java 
b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 2185c6c..72ee270 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -608,24 +608,45 @@ public class SystemKeyspace
         }
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL_CF);
 
-        String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_CF, 
LOCAL_KEY));
-
-        if (result.isEmpty() || !result.one().has("cluster_name"))
         {
-            // this is a brand new node
-            if (!cfs.getSSTables().isEmpty())
-                throw new ConfigurationException("Found system keyspace files, 
but they couldn't be loaded!");
+            String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
+            UntypedResultSet result = executeInternal(String.format(req, 
LOCAL_CF, LOCAL_KEY));
 
-            // no system files.  this is a new node.
-            req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', ?)";
-            executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), 
DatabaseDescriptor.getClusterName());
-            return;
+            if (result.isEmpty() || !result.one().has("cluster_name"))
+            {
+                // this is a brand new node
+                if (!cfs.getSSTables().isEmpty())
+                    throw new ConfigurationException("Found system keyspace 
files, but they couldn't be loaded!");
+
+                // no system files.  this is a new node.
+                req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', 
?)";
+                executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), 
DatabaseDescriptor.getClusterName());
+            }
+            else
+            {
+                String savedClusterName = 
result.one().getString("cluster_name");
+                if 
(!DatabaseDescriptor.getClusterName().equals(savedClusterName))
+                    throw new ConfigurationException("Saved cluster name " + 
savedClusterName + " != configured name " + 
DatabaseDescriptor.getClusterName());
+            }
         }
 
-        String savedClusterName = result.one().getString("cluster_name");
-        if (!DatabaseDescriptor.getClusterName().equals(savedClusterName))
-            throw new ConfigurationException("Saved cluster name " + 
savedClusterName + " != configured name " + 
DatabaseDescriptor.getClusterName());
+        if (!Boolean.getBoolean("cassandra.ignore_rack"))
+        {
+            String req = "SELECT rack FROM system.%s WHERE key='%s'";
+            UntypedResultSet result = executeInternal(String.format(req, 
LOCAL_CF, LOCAL_KEY));
+
+            // Look up the Rack (return it if found)
+            if (!result.isEmpty() && result.one().has("rack"))
+            {
+                String storedRack = result.one().getString("rack");
+                String currentRack = 
DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress());
+                if (!storedRack.equals(currentRack))
+                {
+                    throw new ConfigurationException("Cannot start node if 
snitch's rack (" + currentRack + ") differs from previous rack (" + storedRack 
+ "). " +
+                                                     "Please fix the snitch or 
wipe and rebootstrap this node.");
+                }
+            }
+        }
     }
 
     public static Collection<Token> getSavedTokens()

Reply via email to