You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sm...@apache.org on 2021/10/13 09:46:19 UTC

[cassandra] branch trunk updated: add default_keyspace_rf and minimum_keyspace_rf configuration options

This is an automated email from the ASF dual-hosted git repository.

smiklosovic pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 5fdadb2  add default_keyspace_rf and minimum_keyspace_rf configuration options
5fdadb2 is described below

commit 5fdadb25f95099b8945d9d9ee11d3e380d3867f4
Author: Sumanth Pasupuleti <su...@gmail.com>
AuthorDate: Mon May 11 16:18:51 2020 +0200

    add default_keyspace_rf and minimum_keyspace_rf configuration options
    
    Patch by Sumanth Pasupuleti; reviewed by Jaydeepkumar Chovatia, Alex Petrov, Stefan Miklosovic and Alexander Zotov for CASSANDRA-14557.
---
 CHANGES.txt                                        |   1 +
 conf/cassandra.yaml                                |  19 ++++
 doc/source/cql/ddl.rst                             |  17 ++-
 .../org/apache/cassandra/auth/AuthKeyspace.java    |   3 +-
 .../config/CassandraRelevantProperties.java        |   2 +
 src/java/org/apache/cassandra/config/Config.java   |   5 +
 .../cassandra/config/DatabaseDescriptor.java       |  40 +++++++
 .../locator/AbstractReplicationStrategy.java       |   6 +
 .../cassandra/locator/NetworkTopologyStrategy.java |  12 +-
 .../apache/cassandra/locator/SimpleStrategy.java   |  10 ++
 .../repair/SystemDistributedKeyspace.java          |   4 +-
 .../apache/cassandra/schema/ReplicationParams.java |   4 +-
 .../apache/cassandra/service/StorageService.java   |  22 ++++
 .../cassandra/service/StorageServiceMBean.java     |   5 +
 src/java/org/apache/cassandra/tools/NodeProbe.java |  20 ++++
 src/java/org/apache/cassandra/tools/NodeTool.java  |   4 +
 .../tools/nodetool/GetDefaultKeyspaceRF.java       |  32 ++++++
 .../tools/nodetool/GetMinimumKeyspaceRF.java       |  33 ++++++
 .../tools/nodetool/SetDefaultKeyspaceRF.java       |  36 ++++++
 .../tools/nodetool/SetMinimumKeyspaceRF.java       |  36 ++++++
 .../apache/cassandra/tracing/TraceKeyspace.java    |   6 +-
 .../cassandra/config/DatabaseDescriptorTest.java   |  29 ++++-
 .../cql3/validation/operations/AlterTest.java      | 125 ++++++++++++++++++---
 .../cql3/validation/operations/CreateTest.java     |  45 +++++++-
 .../tools/nodetool/GetDefaultKeyspaceRFTest.java   |  87 ++++++++++++++
 .../tools/nodetool/GetMinimumKeyspaceRFTest.java   |  87 ++++++++++++++
 .../tools/nodetool/SetDefaultKeyspaceRFTest.java   |  94 ++++++++++++++++
 .../tools/nodetool/SetMinimumKeyspaceRFTest.java   |  94 ++++++++++++++++
 28 files changed, 847 insertions(+), 31 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index d910c80..384576b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * Add configuration options for minimum allowable replication factor and default replication factor (CASSANDRA-14557)
  * Expose information about stored hints via a nodetool command and a virtual table (CASSANDRA-14795)
  * Add broadcast_rpc_address to system.local (CASSANDRA-11181)
  * Add support for type casting in WHERE clause components and in the values of INSERT/UPDATE statements (CASSANDRA-14337)
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index fb0263d..a9efa8d 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -1329,6 +1329,25 @@ compaction_tombstone_warning_threshold: 100000
 # as corrupted. This should be positive and less than 2048.
 # max_value_size_in_mb: 256
 
+# ** Impact on keyspace creation **
+# If replication factor is not mentioned as part of keyspace creation, default_keyspace_rf would apply.
+# Changing this configuration would only take effect for keyspaces created after the change, but does not impact
+# existing keyspaces created prior to the change.
+# ** Impact on keyspace alter **
+# When altering a keyspace from NetworkTopologyStrategy to SimpleStrategy, default_keyspace_rf is applied if rf is not
+# explicitly mentioned.
+# ** Impact on system keyspaces **
+# This would also apply for any system keyspaces that need replication factor.
+# A further note about system keyspaces - system_traces and system_distributed keyspaces take RF of 2 or default,
+# whichever is higher, and system_auth keyspace takes RF of 1 or default, whichever is higher.
+# Suggested value for use in production: 3
+# default_keyspace_rf: 1
+
+# The minimum allowable replication factor. Creating a keyspace with a replication factor less than this value will be rejected.
+# This would also apply to system keyspaces.
+# Suggested value for use in production: 2 or higher
+# minimum_keyspace_rf: 0
+
 # Coalescing Strategies #
 # Coalescing multiples messages turns out to significantly boost message processing throughput (think doubling or more).
 # On bare metal, the floor for packet processing throughput is high enough that many applications won't notice, but in
diff --git a/doc/source/cql/ddl.rst b/doc/source/cql/ddl.rst
index 2f442e9..7b69e91 100644
--- a/doc/source/cql/ddl.rst
+++ b/doc/source/cql/ddl.rst
@@ -83,13 +83,17 @@ For instance::
 Attempting to create a keyspace that already exists will return an error unless the ``IF NOT EXISTS`` option is used. If
 it is used, the statement will be a no-op if the keyspace already exists.
 
+Attempting to create a keyspace with a replication factor less than what is configured for ``minimum_keyspace_rf``
+(default value 0) will return an error.
+
 The supported ``options`` are:
 
 =================== ========== =========== ========= ===================================================================
 name                 kind       mandatory   default   description
 =================== ========== =========== ========= ===================================================================
-``replication``      *map*      yes                   The replication strategy and options to use for the keyspace (see
-                                                      details below).
+``replication``      *map*      yes (nts)             The replication strategy and options to use for the keyspace (see
+                                no  (ss)              details below). NOT mandatory for Simple Strategy (ss). This is
+                                                      however mandatory for NetworkTopology Strategy (nts).
 ``durable_writes``   *simple*   no          true      Whether to use the commit log for updates on this keyspace
                                                       (disable this option at your own risk!).
 =================== ========== =========== ========= ===================================================================
@@ -107,12 +111,14 @@ A simple strategy that defines a replication factor for data to be spread
 across the entire cluster. This is generally not a wise choice for production
 because it does not respect datacenter layouts and can lead to wildly varying
 query latency. For a production ready strategy, see
-``NetworkTopologyStrategy``. ``SimpleStrategy`` supports a single mandatory argument:
+``NetworkTopologyStrategy``. ``SimpleStrategy`` supports a single optional argument:
 
 ========================= ====== ======= =============================================
 sub-option                 type   since   description
 ========================= ====== ======= =============================================
-``'replication_factor'``   int    all     The number of replicas to store per range
+``'replication_factor'``   int    all     The number of replicas to store per range.
+                                          If not specified, ``'default_keyspace_rf'``
+                                          would be applied.
 ========================= ====== ======= =============================================
 
 ``NetworkTopologyStrategy``
@@ -134,6 +140,8 @@ sub-option                             type   since  description
                                                      definitions or explicit datacenter settings.
                                                      For example, to have three replicas per
                                                      datacenter, supply this with a value of 3.
+                                                     If not specified, ``'default_keyspace_rf'``
+                                                     would be applied.
 ===================================== ====== ====== =============================================
 
 Note that when ``ALTER`` ing keyspaces and supplying ``replication_factor``,
@@ -175,6 +183,7 @@ For instance, this keyspace will have 3 replicas in DC1, 1 of which is transient
     CREATE KEYSPACE some_keysopace
                WITH replication = {'class': 'NetworkTopologyStrategy', 'DC1' : '3/1'', 'DC2' : '5/2'};
 
+
 .. _use-statement:
 
 USE
diff --git a/src/java/org/apache/cassandra/auth/AuthKeyspace.java b/src/java/org/apache/cassandra/auth/AuthKeyspace.java
index 2271eff..67fc9c1 100644
--- a/src/java/org/apache/cassandra/auth/AuthKeyspace.java
+++ b/src/java/org/apache/cassandra/auth/AuthKeyspace.java
@@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.SchemaConstants;
@@ -112,7 +113,7 @@ public final class AuthKeyspace
     public static KeyspaceMetadata metadata()
     {
         return KeyspaceMetadata.create(SchemaConstants.AUTH_KEYSPACE_NAME,
-                                       KeyspaceParams.simple(DEFAULT_RF),
+                                       KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())),
                                        Tables.of(Roles, RoleMembers, RolePermissions, ResourceRoleIndex, NetworkPermissions));
     }
 }
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
index b4a20da..9e9dd03 100644
--- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -211,6 +211,8 @@ public enum CassandraRelevantProperties
     BATCH_COMMIT_LOG_SYNC_INTERVAL("cassandra.batch_commitlog_sync_interval_millis", "1000"),
 
     SYSTEM_AUTH_DEFAULT_RF("cassandra.system_auth.default_rf", "1"),
+    SYSTEM_TRACES_DEFAULT_RF("cassandra.system_traces.default_rf", "2"),
+    SYSTEM_DISTRIBUTED_DEFAULT_RF("cassandra.system_distributed.default_rf", "3"),
 
     MEMTABLE_OVERHEAD_SIZE("cassandra.memtable.row_overhead_size", "-1"),
     MEMTABLE_OVERHEAD_COMPUTE_STEPS("cassandra.memtable_row_overhead_computation_step", "100000"),
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index f7c0635..3b574ac 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -488,6 +488,11 @@ public class Config
 
     public volatile boolean diagnostic_events_enabled = false;
 
+    // Default and minimum keyspace replication factors allow validation of newly created keyspaces
+    // and good defaults if no replication factor is provided by the user
+    public volatile int default_keyspace_rf = 1;
+    public volatile int minimum_keyspace_rf = 0;
+
     /**
      * flags for enabling tracking repaired state of data during reads
      * separate flags for range & single partition reads as single partition reads are only tracked
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 3341b43..5195646 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -825,6 +825,12 @@ public class DatabaseDescriptor
         }
 
         validateMaxConcurrentAutoUpgradeTasksConf(conf.max_concurrent_automatic_sstable_upgrades);
+
+        if (conf.default_keyspace_rf < conf.minimum_keyspace_rf)
+        {
+            throw new ConfigurationException(String.format("default_keyspace_rf (%d) cannot be less than minimum_keyspace_rf (%d)",
+                                                           conf.default_keyspace_rf, conf.minimum_keyspace_rf));
+        }
     }
 
     @VisibleForTesting
@@ -3539,4 +3545,38 @@ public class DatabaseDescriptor
     {
         conf.track_warnings.row_index_size.setAbortThresholdKb(value);
     }
+
+    public static int getDefaultKeyspaceRF() { return conf.default_keyspace_rf; }
+
+    public static void setDefaultKeyspaceRF(int value) throws ConfigurationException
+    {
+        if (value < 1)
+        {
+            throw new ConfigurationException("default_keyspace_rf cannot be less than 1");
+        }
+
+        if (value < getMinimumKeyspaceRF())
+        {
+            throw new ConfigurationException(String.format("default_keyspace_rf to be set (%d) cannot be less than minimum_keyspace_rf (%d)", value, getMinimumKeyspaceRF()));
+        }
+
+        conf.default_keyspace_rf = value;
+    }
+
+    public static int getMinimumKeyspaceRF() { return conf.minimum_keyspace_rf; }
+
+    public static void setMinimumKeyspaceRF(int value) throws ConfigurationException
+    {
+        if (value < 0)
+        {
+            throw new ConfigurationException("minimum_keyspace_rf cannot be negative");
+        }
+
+        if (value > getDefaultKeyspaceRF())
+        {
+            throw new ConfigurationException(String.format("minimum_keyspace_rf to be set (%d) cannot be greater than default_keyspace_rf (%d)", value, getDefaultKeyspaceRF()));
+        }
+
+        conf.minimum_keyspace_rf = value;
+    }
 }
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index 7891895..e6db054 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -433,6 +433,12 @@ public abstract class AbstractReplicationStrategy
         try
         {
             ReplicationFactor rf = ReplicationFactor.fromString(s);
+
+            if (rf.fullReplicas < DatabaseDescriptor.getMinimumKeyspaceRF())
+            {
+                throw new ConfigurationException(String.format("Replication factor cannot be less than minimum_keyspace_rf (%d), found %d", DatabaseDescriptor.getMinimumKeyspaceRF(), rf.fullReplicas));
+            }
+
             if (rf.hasTransientReplicas())
             {
                 if (DatabaseDescriptor.getNumTokens() > 1)
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 9029dc1..ff88fce 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.locator;
 import java.util.*;
 import java.util.Map.Entry;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -260,11 +261,20 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
      * the "replication_factor" options out into the known datacenters. It is called via reflection from
      * {@link AbstractReplicationStrategy#prepareReplicationStrategyOptions(Class, Map, Map)}.
      *
-     * @param options The proposed strategy options that will be potentially mutated
+     * @param options The proposed strategy options that will be potentially mutated. If empty, replication_factor will
+     *                be added either from previousOptions if one exists, or from default_keyspace_rf configuration.
      * @param previousOptions Any previous strategy options in the case of an ALTER statement
      */
     protected static void prepareOptions(Map<String, String> options, Map<String, String> previousOptions)
     {
+        // add replication_factor only if there is no explicit mention of DCs. Otherwise, non-mentioned DCs will be added with default RF
+        if (options.isEmpty())
+        {
+            String rf = previousOptions.containsKey(REPLICATION_FACTOR) ? previousOptions.get(REPLICATION_FACTOR)
+                                                                        : Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF());
+            options.putIfAbsent(REPLICATION_FACTOR, rf);
+        }
+
         String replication = options.remove(REPLICATION_FACTOR);
 
         if (replication == null && options.size() == 0)
diff --git a/src/java/org/apache/cassandra/locator/SimpleStrategy.java b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
index 928ac97..1a578e9 100644
--- a/src/java/org/apache/cassandra/locator/SimpleStrategy.java
+++ b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -123,4 +124,13 @@ public class SimpleStrategy extends AbstractReplicationStrategy
     {
         return Collections.singleton(REPLICATION_FACTOR);
     }
+
+    protected static void prepareOptions(Map<String, String> options, Map<String, String> previousOptions)
+    {
+        // When altering from NTS to SS, previousOptions could have multiple different RFs for different data centers - so we
+        // will instead default to DefaultRF configuration if RF is not mentioned with the alter statement
+        String rf = previousOptions.containsKey(REPLICATION_FACTOR) ? previousOptions.get(REPLICATION_FACTOR)
+                                                                    : Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF());
+        options.putIfAbsent(REPLICATION_FACTOR, rf);
+    }
 }
diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
index f0faba1..5df38a7 100644
--- a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
+++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
@@ -38,6 +38,7 @@ import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -68,6 +69,7 @@ public final class SystemDistributedKeyspace
     {
     }
 
+    private static final int DEFAULT_RF = CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF.getInt();
     private static final Logger logger = LoggerFactory.getLogger(SystemDistributedKeyspace.class);
 
     /**
@@ -154,7 +156,7 @@ public final class SystemDistributedKeyspace
 
     public static KeyspaceMetadata metadata()
     {
-        return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(3), Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus));
+        return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())), Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus));
     }
 
     public static void startParentRepair(UUID parent_id, String keyspaceName, String[] cfnames, RepairOption options)
diff --git a/src/java/org/apache/cassandra/schema/ReplicationParams.java b/src/java/org/apache/cassandra/schema/ReplicationParams.java
index 048b4ed..8fbf1cd 100644
--- a/src/java/org/apache/cassandra/schema/ReplicationParams.java
+++ b/src/java/org/apache/cassandra/schema/ReplicationParams.java
@@ -82,13 +82,13 @@ public final class ReplicationParams
         return fromMapWithDefaults(map, new HashMap<>());
     }
 
-    public static ReplicationParams fromMapWithDefaults(Map<String, String> map, Map<String, String> defaults)
+    public static ReplicationParams fromMapWithDefaults(Map<String, String> map, Map<String, String> previousOptions)
     {
         Map<String, String> options = new HashMap<>(map);
         String className = options.remove(CLASS);
 
         Class<? extends AbstractReplicationStrategy> klass = AbstractReplicationStrategy.getClass(className);
-        AbstractReplicationStrategy.prepareReplicationStrategyOptions(klass, options, defaults);
+        AbstractReplicationStrategy.prepareReplicationStrategyOptions(klass, options, previousOptions);
 
         return new ReplicationParams(klass, options);
     }
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 93bf478..65a614a 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -6269,4 +6269,26 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         DatabaseDescriptor.setRowIndexSizeAbortThresholdKb(value);
         logger.info("updated track_warnings.row_index_size.abort_threshold_kb to {}", value);
     }
+
+    public void setDefaultKeyspaceReplicationFactor(int value)
+    {
+        DatabaseDescriptor.setDefaultKeyspaceRF(value);
+        logger.info("set default keyspace rf to {}", value);
+    }
+
+    public int getDefaultKeyspaceReplicationFactor()
+    {
+        return DatabaseDescriptor.getDefaultKeyspaceRF();
+    }
+
+    public void setMinimumKeyspaceReplicationFactor(int value)
+    {
+        DatabaseDescriptor.setMinimumKeyspaceRF(value);
+        logger.info("set minimum keyspace rf to {}", value);
+    }
+
+    public int getMinimumKeyspaceReplicationFactor()
+    {
+        return DatabaseDescriptor.getMinimumKeyspaceRF();
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 6a294c1..256083f 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -909,4 +909,9 @@ public interface StorageServiceMBean extends NotificationEmitter
     public void setRowIndexSizeWarnThresholdKb(int value);
     public int getRowIndexSizeAbortThresholdKb();
     public void setRowIndexSizeAbortThresholdKb(int value);
+
+    public void setDefaultKeyspaceReplicationFactor(int value);
+    public int getDefaultKeyspaceReplicationFactor();
+    public void setMinimumKeyspaceReplicationFactor(int value);
+    public int getMinimumKeyspaceReplicationFactor();
 }
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index ae50a68..46481b4 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -1968,6 +1968,26 @@ public class NodeProbe implements AutoCloseable
     {
         return AuditLogOptionsCompositeData.fromCompositeData(almProxy.getAuditLogOptionsData());
     }
+
+    public void setDefaultKeyspaceReplicationFactor(int value)
+    {
+        ssProxy.setDefaultKeyspaceReplicationFactor(value);
+    }
+
+    public int getDefaultKeyspaceReplicationFactor()
+    {
+        return ssProxy.getDefaultKeyspaceReplicationFactor();
+    }
+
+    public void setMinimumKeyspaceReplicationFactor(int value)
+    {
+        ssProxy.setMinimumKeyspaceReplicationFactor(value);
+    }
+
+    public int getMinimumKeyspaceReplicationFactor()
+    {
+        return ssProxy.getMinimumKeyspaceReplicationFactor();
+    }
 }
 
 class ColumnFamilyStoreMBeanIterator implements Iterator<Map.Entry<String, ColumnFamilyStoreMBean>>
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 63e6eb6..f3a8838 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -136,11 +136,13 @@ public class NodeTool
                 GetConcurrency.class,
                 GetConcurrentCompactors.class,
                 GetConcurrentViewBuilders.class,
+                GetDefaultKeyspaceRF.class,
                 GetEndpoints.class,
                 GetFullQueryLog.class,
                 GetInterDCStreamThroughput.class,
                 GetLoggingLevels.class,
                 GetMaxHintWindow.class,
+                GetMinimumKeyspaceRF.class,
                 GetSSTables.class,
                 GetSeeds.class,
                 GetSnapshotThrottle.class,
@@ -193,10 +195,12 @@ public class NodeTool
                 SetConcurrency.class,
                 SetConcurrentCompactors.class,
                 SetConcurrentViewBuilders.class,
+                SetDefaultKeyspaceRF.class,
                 SetHintedHandoffThrottleInKB.class,
                 SetInterDCStreamThroughput.class,
                 SetLoggingLevel.class,
                 SetMaxHintWindow.class,
+                SetMinimumKeyspaceRF.class,
                 SetSnapshotThrottle.class,
                 SetStreamThroughput.class,
                 SetTimeout.class,
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetDefaultKeyspaceRF.java b/src/java/org/apache/cassandra/tools/nodetool/GetDefaultKeyspaceRF.java
new file mode 100644
index 0000000..0ba7d37
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetDefaultKeyspaceRF.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.nodetool;
+
+import io.airlift.airline.Command;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+
+@Command(name = "getdefaultrf", description = "Gets default keyspace replication factor.")
+public class GetDefaultKeyspaceRF extends NodeTool.NodeToolCmd
+{
+    protected void execute(NodeProbe probe)
+    {
+        probe.output().out.println(probe.getDefaultKeyspaceReplicationFactor());
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRF.java b/src/java/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRF.java
new file mode 100644
index 0000000..2de94f5
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRF.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.nodetool;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+
+@Command(name = "getminimumrf", description = "Gets minimum keyspace replication factor.")
+public class GetMinimumKeyspaceRF extends NodeTool.NodeToolCmd
+{
+    protected void execute(NodeProbe probe)
+    {
+        probe.output().out.println(probe.getMinimumKeyspaceReplicationFactor());
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetDefaultKeyspaceRF.java b/src/java/org/apache/cassandra/tools/nodetool/SetDefaultKeyspaceRF.java
new file mode 100644
index 0000000..6126bac
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetDefaultKeyspaceRF.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.nodetool;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+
+@Command(name = "setdefaultrf", description = "Sets default keyspace replication factor.")
+public class SetDefaultKeyspaceRF extends NodeTool.NodeToolCmd
+{
+    @Arguments(title = "default_rf", usage = "<value>", description = "Default replication factor", required = true)
+    private Integer defaultRF = null;
+
+    protected void execute(NodeProbe probe)
+    {
+        probe.setDefaultKeyspaceReplicationFactor(defaultRF);
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetMinimumKeyspaceRF.java b/src/java/org/apache/cassandra/tools/nodetool/SetMinimumKeyspaceRF.java
new file mode 100644
index 0000000..92ab7b7
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetMinimumKeyspaceRF.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.nodetool;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+
+@Command(name = "setminimumrf", description = "Sets minimum keyspace replication factor.")
+public class SetMinimumKeyspaceRF extends NodeTool.NodeToolCmd
+{
+    @Arguments(title = "minimum_rf", usage = "<value>", description = "Minimum replication factor", required = true)
+    private Integer minimumRF = null;
+
+    protected void execute(NodeProbe probe)
+    {
+        probe.setMinimumKeyspaceReplicationFactor(minimumRF);
+    }
+}
diff --git a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
index cd3fa8a..1edeccf 100644
--- a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
+++ b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
@@ -21,7 +21,9 @@ import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.Row;
@@ -43,6 +45,8 @@ public final class TraceKeyspace
     {
     }
 
+    private static final int DEFAULT_RF = CassandraRelevantProperties.SYSTEM_TRACES_DEFAULT_RF.getInt();
+
     /**
      * Generation is used as a timestamp for automatic table creation on startup.
      * If you make any changes to the tables below, make sure to increment the
@@ -102,7 +106,7 @@ public final class TraceKeyspace
 
     public static KeyspaceMetadata metadata()
     {
-        return KeyspaceMetadata.create(SchemaConstants.TRACE_KEYSPACE_NAME, KeyspaceParams.simple(2), Tables.of(Sessions, Events));
+        return KeyspaceMetadata.create(SchemaConstants.TRACE_KEYSPACE_NAME, KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())), Tables.of(Sessions, Events));
     }
 
     static Mutation makeStartSessionMutation(ByteBuffer sessionId,
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 8536c01..065f557 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -743,7 +743,8 @@ public class DatabaseDescriptorTest
     }
 
     @Test
-    public void testDefaultSslContextFactoryConfiguration() {
+    public void testDefaultSslContextFactoryConfiguration()
+    {
         Config config = DatabaseDescriptor.loadConfig();
         Assert.assertEquals("org.apache.cassandra.security.DefaultSslContextFactory",
                             config.client_encryption_options.ssl_context_factory.class_name);
@@ -752,4 +753,30 @@ public class DatabaseDescriptorTest
                             config.server_encryption_options.ssl_context_factory.class_name);
         Assert.assertTrue(config.server_encryption_options.ssl_context_factory.parameters.isEmpty());
     }
+
+    @Test (expected = ConfigurationException.class)
+    public void testInvalidSub1DefaultRFs() throws ConfigurationException
+    {
+        DatabaseDescriptor.setDefaultKeyspaceRF(0);
+    }
+
+    @Test (expected = ConfigurationException.class)
+    public void testInvalidSub0MinimumRFs() throws ConfigurationException
+    {
+        DatabaseDescriptor.setMinimumKeyspaceRF(-1);
+    }
+
+    @Test (expected = ConfigurationException.class)
+    public void testDefaultRfLessThanMinRF()
+    {
+        DatabaseDescriptor.setMinimumKeyspaceRF(2);
+        DatabaseDescriptor.setDefaultKeyspaceRF(1);
+    }
+
+    @Test (expected = ConfigurationException.class)
+    public void testMinimumRfGreaterThanDefaultRF()
+    {
+        DatabaseDescriptor.setDefaultKeyspaceRF(1);
+        DatabaseDescriptor.setMinimumKeyspaceRF(2);
+    }
 }
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 401168a..feddab3 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -22,6 +22,11 @@ import java.util.UUID;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.OrderPreservingPartitioner;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -340,27 +345,115 @@ public class AlterTest extends CQLTester
                                         row(ks1, true, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER, "2", DATA_CENTER_REMOTE, "2")));
     }
 
-    /**
-     * Test {@link ConfigurationException} thrown on alter keyspace to no DC option in replication configuration.
-     */
     @Test
-    public void testAlterKeyspaceWithNoOptionThrowsConfigurationException() throws Throwable
+    public void testDefaultRF() throws Throwable
+    {
+        TokenMetadata metadata = StorageService.instance.getTokenMetadata();
+        metadata.clearUnsafe();
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
+        InetAddressAndPort remote = InetAddressAndPort.getByName("127.0.0.4");
+        metadata.updateHostId(UUID.randomUUID(), local);
+        metadata.updateNormalToken(new OrderPreservingPartitioner.StringToken("A"), local);
+        metadata.updateHostId(UUID.randomUUID(), remote);
+        metadata.updateNormalToken(new OrderPreservingPartitioner.StringToken("B"), remote);
+
+        DatabaseDescriptor.setDefaultKeyspaceRF(3);
+
+        //ensure default rf is being taken into account during creation, and user can choose to override the default
+        String ks1 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy' }");
+        String ks2 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
+        String ks3 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'NetworkTopologyStrategy' }");
+        String ks4 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'NetworkTopologyStrategy', 'replication_factor' : 2 }");
+
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks1, true, map("class","org.apache.cassandra.locator.SimpleStrategy","replication_factor", Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF()))),
+                                        row(ks2, true, map("class","org.apache.cassandra.locator.SimpleStrategy","replication_factor", "2")),
+                                        row(ks3, true, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER,
+                                                           Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF()), DATA_CENTER_REMOTE, Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF()))),
+                                        row(ks4, true, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER, "2", DATA_CENTER_REMOTE, "2")));
+
+        //ensure alter keyspace does not default to default rf unless altering from NTS to SS
+        //no change alter
+        schemaChange("ALTER KEYSPACE " + ks4 + " WITH durable_writes=true");
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks4, true, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER, "2", DATA_CENTER_REMOTE, "2")));
+        schemaChange("ALTER KEYSPACE " + ks4 + " WITH replication={ 'class' : 'NetworkTopologyStrategy' }");
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks4, true, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER, "2", DATA_CENTER_REMOTE, "2")));
+
+        // change from SS to NTS
+        // without specifying RF
+        schemaChange("ALTER KEYSPACE " + ks2 + " WITH replication={ 'class' : 'NetworkTopologyStrategy' } AND durable_writes=true");
+        // verify that RF of SS is retained
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks2, true, map("class","org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER, "2", DATA_CENTER_REMOTE, "2")));
+        // with specifying RF
+        schemaChange("ALTER KEYSPACE " + ks1 + " WITH replication={ 'class' : 'NetworkTopologyStrategy', 'replication_factor': '1' } AND durable_writes=true");
+        // verify that explicitly mentioned RF is taken into account
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks1, true, map("class","org.apache.cassandra.locator.NetworkTopologyStrategy", DATA_CENTER, "1", DATA_CENTER_REMOTE, "1")));
+
+        // change from NTS to SS
+        // without specifying RF
+        schemaChange("ALTER KEYSPACE " + ks4 + " WITH replication={ 'class' : 'SimpleStrategy' } AND durable_writes=true");
+        // verify that default RF is taken into account
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks4, true, map("class","org.apache.cassandra.locator.SimpleStrategy","replication_factor", Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF()))));
+        // with specifying RF
+        schemaChange("ALTER KEYSPACE " + ks3 + " WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : '1' } AND durable_writes=true");
+        // verify that explicitly mentioned RF is taken into account
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks3, true, map("class","org.apache.cassandra.locator.SimpleStrategy","replication_factor", "1")));
+
+        // verify updated default does not effect existing keyspaces
+        // create keyspaces
+        String ks5 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy' }");
+        String ks6 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'NetworkTopologyStrategy' }");
+        String oldRF = Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF());
+        // change default
+        DatabaseDescriptor.setDefaultKeyspaceRF(2);
+        // verify RF of existing keyspaces
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks5, true, map("class","org.apache.cassandra.locator.SimpleStrategy","replication_factor", oldRF)));
+        assertRowsIgnoringOrderAndExtra(execute("SELECT keyspace_name, durable_writes, replication FROM system_schema.keyspaces"),
+                                        row(ks6, true, map("class", "org.apache.cassandra.locator.NetworkTopologyStrategy",
+                                                           DATA_CENTER, oldRF, DATA_CENTER_REMOTE, oldRF)));
+
+        //clean up config change
+        DatabaseDescriptor.setDefaultKeyspaceRF(1);
+
+        //clean up keyspaces
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", ks1));
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", ks2));
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", ks3));
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", ks4));
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", ks5));
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", ks6));
+    }
+
+    @Test
+    public void testMinimumRF() 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 }");
+        DatabaseDescriptor.setDefaultKeyspaceRF(3);
+        DatabaseDescriptor.setMinimumKeyspaceRF(2);
+
+        String ks1 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy' }");
+        String ks2 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'NetworkTopologyStrategy' }");
 
-        // 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' }");
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        String.format("Replication factor cannot be less than minimum_keyspace_rf (%s), found %s", DatabaseDescriptor.getMinimumKeyspaceRF(), "1"),
+                                        String.format("ALTER KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }", ks1));
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        String.format("Replication factor cannot be less than minimum_keyspace_rf (%s), found %s", DatabaseDescriptor.getMinimumKeyspaceRF(), "1"),
+                                        String.format("ALTER KEYSPACE %s WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : '1' }", ks2));
 
-        // Make sure that the alter works as expected
-        alterTable("ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-        alterTable("ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
+        //clean up config change
+        DatabaseDescriptor.setMinimumKeyspaceRF(0);
+        DatabaseDescriptor.setDefaultKeyspaceRF(1);
 
-        // clean up
-        execute("DROP KEYSPACE IF EXISTS testABC");
-        execute("DROP KEYSPACE IF EXISTS testXYZ");
+        //clean up keyspaces
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", ks1));
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", ks2));
     }
 
     /**
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 3d5c680..e2a8bab 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -388,13 +388,27 @@ public class CreateTest extends CQLTester
     }
 
     /**
-     * Test {@link ConfigurationException} is thrown on create keyspace without any options.
+     * Test {@link ConfigurationException} is not thrown on create NetworkTopologyStrategy keyspace without any options.
      */
     @Test
-    public void testConfigurationExceptionThrownWhenCreateKeyspaceWithNoOptions() throws Throwable
+    public void testCreateKeyspaceWithNetworkTopologyStrategyNoOptions() throws Throwable
     {
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ with replication = { 'class': 'NetworkTopologyStrategy' }");
-        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy' }");
+        schemaChange("CREATE KEYSPACE testXYZ with replication = { 'class': 'NetworkTopologyStrategy' }");
+
+        // clean-up
+        execute("DROP KEYSPACE IF EXISTS testXYZ");
+    }
+
+    /**
+     * Test {@link ConfigurationException} is not thrown on create SimpleStrategy keyspace without any options.
+     */
+    @Test
+    public void testCreateKeyspaceWithSimpleStrategyNoOptions() throws Throwable
+    {
+        schemaChange("CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy' }");
+
+        // clean-up
+        execute("DROP KEYSPACE IF EXISTS testXYZ");
     }
 
     @Test
@@ -723,4 +737,27 @@ public class CreateTest extends CQLTester
             return Collections.emptyList();
         }
     }
+
+    @Test
+    public void testMinimumRF()
+    {
+        try
+        {
+            DatabaseDescriptor.setDefaultKeyspaceRF(3);
+            DatabaseDescriptor.setMinimumKeyspaceRF(2);
+
+            assertThrowsConfigurationException(
+            String.format("Replication factor cannot be less than minimum_keyspace_rf (%s), found %s", DatabaseDescriptor.getMinimumKeyspaceRF(), "1"),
+            "CREATE KEYSPACE ks1 WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+
+            assertThrowsConfigurationException(
+            String.format("Replication factor cannot be less than minimum_keyspace_rf (%s), found %s", DatabaseDescriptor.getMinimumKeyspaceRF(), "1"),
+            "CREATE KEYSPACE ks2 WITH replication={ 'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1 }");
+        }
+        finally
+        {
+            DatabaseDescriptor.setMinimumKeyspaceRF(0);
+            DatabaseDescriptor.setDefaultKeyspaceRF(1);
+        }
+    }
 }
diff --git a/test/unit/org/apache/cassandra/tools/nodetool/GetDefaultKeyspaceRFTest.java b/test/unit/org/apache/cassandra/tools/nodetool/GetDefaultKeyspaceRFTest.java
new file mode 100644
index 0000000..9e93e1a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/nodetool/GetDefaultKeyspaceRFTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.nodetool;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.tools.ToolRunner;
+import org.assertj.core.api.Assertions;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class GetDefaultKeyspaceRFTest extends CQLTester
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        requireNetwork();
+        startJMXServer();
+    }
+
+    @Test
+    @SuppressWarnings("SingleCharacterStringConcatenation")
+    public void testMaybeChangeDocs()
+    {
+        // If you added, modified options or help, please update docs if necessary
+        ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("help", "getdefaultrf");
+        tool.assertOnCleanExit();
+
+        String help =   "NAME\n" +
+                        "        nodetool getdefaultrf - Gets default keyspace replication factor.\n" +
+                        "\n" +
+                        "SYNOPSIS\n" +
+                        "        nodetool [(-h <host> | --host <host>)] [(-p <port> | --port <port>)]\n" +
+                        "                [(-pp | --print-port)] [(-pw <password> | --password <password>)]\n" +
+                        "                [(-pwf <passwordFilePath> | --password-file <passwordFilePath>)]\n" +
+                        "                [(-u <username> | --username <username>)] getdefaultrf\n" +
+                        "\n" +
+                        "OPTIONS\n" +
+                        "        -h <host>, --host <host>\n" +
+                        "            Node hostname or ip address\n" +
+                        "\n" +
+                        "        -p <port>, --port <port>\n" +
+                        "            Remote jmx agent port number\n" +
+                        "\n" +
+                        "        -pp, --print-port\n" +
+                        "            Operate in 4.0 mode with hosts disambiguated by port number\n" +
+                        "\n" +
+                        "        -pw <password>, --password <password>\n" +
+                        "            Remote jmx agent password\n" +
+                        "\n" +
+                        "        -pwf <passwordFilePath>, --password-file <passwordFilePath>\n" +
+                        "            Path to the JMX password file\n" +
+                        "\n" +
+                        "        -u <username>, --username <username>\n" +
+                        "            Remote jmx agent username\n" +
+                        "\n" +
+                        "\n";
+        assertThat(tool.getStdout()).isEqualTo(help);
+    }
+
+    @Test
+    public void testGetDefaultRF()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("getdefaultrf");
+        tool.assertOnCleanExit();
+        assertThat(tool.getStdout().trim()).isEqualTo(Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF()));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRFTest.java b/test/unit/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRFTest.java
new file mode 100644
index 0000000..d2ff34b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRFTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.nodetool;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.tools.ToolRunner;
+import org.assertj.core.api.Assertions;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class GetMinimumKeyspaceRFTest extends CQLTester
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        requireNetwork();
+        startJMXServer();
+    }
+
+    @Test
+    @SuppressWarnings("SingleCharacterStringConcatenation")
+    public void testMaybeChangeDocs()
+    {
+        // If you added, modified options or help, please update docs if necessary
+        ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("help", "getminimumrf");
+        tool.assertOnCleanExit();
+
+        String help =   "NAME\n" +
+                        "        nodetool getminimumrf - Gets minimum keyspace replication factor.\n" +
+                        "\n" +
+                        "SYNOPSIS\n" +
+                        "        nodetool [(-h <host> | --host <host>)] [(-p <port> | --port <port>)]\n" +
+                        "                [(-pp | --print-port)] [(-pw <password> | --password <password>)]\n" +
+                        "                [(-pwf <passwordFilePath> | --password-file <passwordFilePath>)]\n" +
+                        "                [(-u <username> | --username <username>)] getminimumrf\n" +
+                        "\n" +
+                        "OPTIONS\n" +
+                        "        -h <host>, --host <host>\n" +
+                        "            Node hostname or ip address\n" +
+                        "\n" +
+                        "        -p <port>, --port <port>\n" +
+                        "            Remote jmx agent port number\n" +
+                        "\n" +
+                        "        -pp, --print-port\n" +
+                        "            Operate in 4.0 mode with hosts disambiguated by port number\n" +
+                        "\n" +
+                        "        -pw <password>, --password <password>\n" +
+                        "            Remote jmx agent password\n" +
+                        "\n" +
+                        "        -pwf <passwordFilePath>, --password-file <passwordFilePath>\n" +
+                        "            Path to the JMX password file\n" +
+                        "\n" +
+                        "        -u <username>, --username <username>\n" +
+                        "            Remote jmx agent username\n" +
+                        "\n" +
+                        "\n";
+        assertThat(tool.getStdout()).isEqualTo(help);
+    }
+
+    @Test
+    public void testGetMinimumRF()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("getminimumrf");
+        tool.assertOnCleanExit();
+        assertThat(tool.getStdout().trim()).isEqualTo(Integer.toString(DatabaseDescriptor.getMinimumKeyspaceRF()));
+    }
+}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetDefaultKeyspaceRFTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetDefaultKeyspaceRFTest.java
new file mode 100644
index 0000000..92250a3
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/nodetool/SetDefaultKeyspaceRFTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.nodetool;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.tools.ToolRunner;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class SetDefaultKeyspaceRFTest extends CQLTester
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        requireNetwork();
+        startJMXServer();
+    }
+
+    @Test
+    @SuppressWarnings("SingleCharacterStringConcatenation")
+    public void testMaybeChangeDocs()
+    {
+        // If you added, modified options or help, please update docs if necessary
+        ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("help", "setdefaultrf");
+        tool.assertOnCleanExit();
+
+        String help =   "NAME\n" +
+                        "        nodetool setdefaultrf - Sets default keyspace replication factor.\n" +
+                        "\n" +
+                        "SYNOPSIS\n" +
+                        "        nodetool [(-h <host> | --host <host>)] [(-p <port> | --port <port>)]\n" +
+                        "                [(-pp | --print-port)] [(-pw <password> | --password <password>)]\n" +
+                        "                [(-pwf <passwordFilePath> | --password-file <passwordFilePath>)]\n" +
+                        "                [(-u <username> | --username <username>)] setdefaultrf [--] <value>\n" +
+                        "\n" +
+                        "OPTIONS\n" +
+                        "        -h <host>, --host <host>\n" +
+                        "            Node hostname or ip address\n" +
+                        "\n" +
+                        "        -p <port>, --port <port>\n" +
+                        "            Remote jmx agent port number\n" +
+                        "\n" +
+                        "        -pp, --print-port\n" +
+                        "            Operate in 4.0 mode with hosts disambiguated by port number\n" +
+                        "\n" +
+                        "        -pw <password>, --password <password>\n" +
+                        "            Remote jmx agent password\n" +
+                        "\n" +
+                        "        -pwf <passwordFilePath>, --password-file <passwordFilePath>\n" +
+                        "            Path to the JMX password file\n" +
+                        "\n" +
+                        "        -u <username>, --username <username>\n" +
+                        "            Remote jmx agent username\n" +
+                        "\n" +
+                        "        --\n" +
+                        "            This option can be used to separate command-line options from the\n" +
+                        "            list of argument, (useful when arguments might be mistaken for\n" +
+                        "            command-line options\n" +
+                        "\n" +
+                        "        <value>\n" +
+                        "            Default replication factor\n" +
+                        "\n" +
+                        "\n";
+        assertThat(tool.getStdout()).isEqualTo(help);
+    }
+
+    @Test
+    public void testSetDefaultRF()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("setdefaultrf", "2");
+        tool.assertOnCleanExit();
+        assertThat(DatabaseDescriptor.getDefaultKeyspaceRF()).isEqualTo(2);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetMinimumKeyspaceRFTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetMinimumKeyspaceRFTest.java
new file mode 100644
index 0000000..09d3625
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/nodetool/SetMinimumKeyspaceRFTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.tools.nodetool;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.tools.ToolRunner;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class SetMinimumKeyspaceRFTest extends CQLTester
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        requireNetwork();
+        startJMXServer();
+    }
+
+    @Test
+    @SuppressWarnings("SingleCharacterStringConcatenation")
+    public void testMaybeChangeDocs()
+    {
+        // If you added, modified options or help, please update docs if necessary
+        ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("help", "setminimumrf");
+        tool.assertOnCleanExit();
+
+        String help =   "NAME\n" +
+                        "        nodetool setminimumrf - Sets minimum keyspace replication factor.\n" +
+                        "\n" +
+                        "SYNOPSIS\n" +
+                        "        nodetool [(-h <host> | --host <host>)] [(-p <port> | --port <port>)]\n" +
+                        "                [(-pp | --print-port)] [(-pw <password> | --password <password>)]\n" +
+                        "                [(-pwf <passwordFilePath> | --password-file <passwordFilePath>)]\n" +
+                        "                [(-u <username> | --username <username>)] setminimumrf [--] <value>\n" +
+                        "\n" +
+                        "OPTIONS\n" +
+                        "        -h <host>, --host <host>\n" +
+                        "            Node hostname or ip address\n" +
+                        "\n" +
+                        "        -p <port>, --port <port>\n" +
+                        "            Remote jmx agent port number\n" +
+                        "\n" +
+                        "        -pp, --print-port\n" +
+                        "            Operate in 4.0 mode with hosts disambiguated by port number\n" +
+                        "\n" +
+                        "        -pw <password>, --password <password>\n" +
+                        "            Remote jmx agent password\n" +
+                        "\n" +
+                        "        -pwf <passwordFilePath>, --password-file <passwordFilePath>\n" +
+                        "            Path to the JMX password file\n" +
+                        "\n" +
+                        "        -u <username>, --username <username>\n" +
+                        "            Remote jmx agent username\n" +
+                        "\n" +
+                        "        --\n" +
+                        "            This option can be used to separate command-line options from the\n" +
+                        "            list of argument, (useful when arguments might be mistaken for\n" +
+                        "            command-line options\n" +
+                        "\n" +
+                        "        <value>\n" +
+                        "            Minimum replication factor\n" +
+                        "\n" +
+                        "\n";
+        assertThat(tool.getStdout()).isEqualTo(help);
+    }
+
+    @Test
+    public void testSetMinimumRF()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("setminimumrf", "1");
+        tool.assertOnCleanExit();
+        assertThat(DatabaseDescriptor.getMinimumKeyspaceRF()).isEqualTo(1);
+    }
+}

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org