cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From marc...@apache.org
Subject cassandra git commit: Output warning if user increases RF
Date Mon, 20 Feb 2017 12:53:07 GMT
Repository: cassandra
Updated Branches:
  refs/heads/trunk eaa594865 -> 27efbf2fc


Output warning if user increases RF

Patch by marcuse; reviewed by Paulo Motta for CASSANDRA-13079


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

Branch: refs/heads/trunk
Commit: 27efbf2fc55ff064767814067ef50396eb2403f0
Parents: eaa5948
Author: Marcus Eriksson <marcuse@apache.org>
Authored: Fri Feb 10 15:04:15 2017 -0800
Committer: Marcus Eriksson <marcuse@apache.org>
Committed: Mon Feb 20 13:52:11 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 doc/source/faq/index.rst                        |  7 ++++---
 .../cql3/statements/AlterKeyspaceStatement.java | 21 ++++++++++++++++++++
 3 files changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/27efbf2f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ed52a77..893dd74 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Output warning if user increases RF (CASSANDRA-13079)
  * Remove pre-3.0 streaming compatibility code for 4.0 (CASSANDRA-13081)
  * Add support for + and - operations on dates (CASSANDRA-11936)
  * Fix consistency of incrementally repaired data (CASSANDRA-9143)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27efbf2f/doc/source/faq/index.rst
----------------------------------------------------------------------
diff --git a/doc/source/faq/index.rst b/doc/source/faq/index.rst
index 24575a4..acb7538 100644
--- a/doc/source/faq/index.rst
+++ b/doc/source/faq/index.rst
@@ -98,15 +98,16 @@ token on the next restart.
 Can I change the replication factor (a a keyspace) on a live cluster?
 ---------------------------------------------------------------------
 
-Yes, but it will require running repair (or cleanup) to change the replica count of existing
data:
+Yes, but it will require running a full repair (or cleanup) to change the replica count of
existing data:
 
 - :ref:`Alter <alter-keyspace-statement>` the replication factor for desired keyspace
(using cqlsh for instance).
 - If you're reducing the replication factor, run ``nodetool cleanup`` on the cluster to remove
surplus replicated data.
   Cleanup runs on a per-node basis.
-- If you're increasing the replication factor, run ``nodetool repair`` to ensure data is
replicated according to the new
+- If you're increasing the replication factor, run ``nodetool repair -full`` to ensure data
is replicated according to the new
   configuration. Repair runs on a per-replica set basis. This is an intensive process that
may result in adverse cluster
   performance. It's highly recommended to do rolling repairs, as an attempt to repair the
entire cluster at once will
-  most likely swamp it.
+  most likely swamp it. Note that you will need to run a full repair (``-full``) to make
sure that already repaired
+  sstables are not skipped.
 
 .. _can-large-blob:
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27efbf2f/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
index 68700f2..0de5b2a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
@@ -18,7 +18,9 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -26,7 +28,9 @@ import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.transport.Event;
 
 public class AlterKeyspaceStatement extends SchemaAlteringStatement
@@ -74,9 +78,26 @@ public class AlterKeyspaceStatement extends SchemaAlteringStatement
             params.validate(name);
             if (params.replication.klass.equals(LocalStrategy.class))
                 throw new ConfigurationException("Unable to use given strategy class: LocalStrategy
is reserved for internal use.");
+            warnIfIncreasingRF(ksm, params);
         }
     }
 
+    private void warnIfIncreasingRF(KeyspaceMetadata ksm, KeyspaceParams params)
+    {
+        AbstractReplicationStrategy oldStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,
+                                                                                        
               ksm.params.replication.klass,
+                                                                                        
               StorageService.instance.getTokenMetadata(),
+                                                                                        
               DatabaseDescriptor.getEndpointSnitch(),
+                                                                                        
               ksm.params.replication.options);
+        AbstractReplicationStrategy newStrategy = AbstractReplicationStrategy.createReplicationStrategy(keyspace(),
+                                                                                        
               params.replication.klass,
+                                                                                        
               StorageService.instance.getTokenMetadata(),
+                                                                                        
               DatabaseDescriptor.getEndpointSnitch(),
+                                                                                        
               params.replication.options);
+        if (newStrategy.getReplicationFactor() > oldStrategy.getReplicationFactor())
+            ClientWarn.instance.warn("When increasing replication factor you need to run
a full (-full) repair to distribute the data.");
+    }
+
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly)
throws RequestValidationException
     {
         KeyspaceMetadata oldKsm = Schema.instance.getKeyspaceMetadata(name);


Mime
View raw message