You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2016/01/15 15:50:37 UTC

[1/5] cassandra git commit: Allow simultaneous bootstrapping with strict consistency when no vnodes are used

Repository: cassandra
Updated Branches:
  refs/heads/trunk 75f78729d -> 428806165


Allow simultaneous bootstrapping with strict consistency when no vnodes are used

patch by Sankalp Kohli; reviewed by tjake for CASSANDRA-11005


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

Branch: refs/heads/trunk
Commit: abe0c6779c287c4f55a48a08c83f9cda7d6616f5
Parents: 03e6f45
Author: T Jake Luciani <ja...@apache.org>
Authored: Fri Jan 15 09:31:22 2016 -0500
Committer: T Jake Luciani <ja...@apache.org>
Committed: Fri Jan 15 09:31:22 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                               | 1 +
 src/java/org/apache/cassandra/dht/RangeStreamer.java      | 1 +
 src/java/org/apache/cassandra/service/StorageService.java | 7 ++++++-
 3 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/abe0c677/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 636c15d..4b87ed0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.13
+ * Allow simultaneous bootstrapping with strict consistency when no vnodes are used (CASSANDRA-11005)
  * Log a message when major compaction does not result in a single file (CASSANDRA-10847)
  * (cqlsh) fix cqlsh_copy_tests when vnodes are disabled (CASSANDRA-10997)
  * (cqlsh) fix formatting bytearray values (CASSANDRA-10839)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/abe0c677/src/java/org/apache/cassandra/dht/RangeStreamer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java
index 3b095b1..f8e29b6 100644
--- a/src/java/org/apache/cassandra/dht/RangeStreamer.java
+++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java
@@ -49,6 +49,7 @@ public class RangeStreamer
 {
     private static final Logger logger = LoggerFactory.getLogger(RangeStreamer.class);
     public static final boolean useStrictConsistency = Boolean.valueOf(System.getProperty("cassandra.consistent.rangemovement","true"));
+    public static final boolean allowSimultaneousMoves = Boolean.valueOf(System.getProperty("cassandra.consistent.simultaneousmoves.allow","false"));
     private final Collection<Token> tokens;
     private final TokenMetadata metadata;
     private final InetAddress address;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/abe0c677/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f134e8a..606670c 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -545,7 +545,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                                      "Use cassandra.replace_address if you want to replace this node.",
                                                      FBUtilities.getBroadcastAddress()));
         }
-        if (RangeStreamer.useStrictConsistency)
+        if (RangeStreamer.useStrictConsistency && !allowSimultaneousMoves())
         {
             for (Map.Entry<InetAddress, EndpointState> entry : Gossiper.instance.getEndpointStates())
             {
@@ -562,6 +562,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         Gossiper.instance.resetEndpointStateMap();
     }
 
+    private boolean allowSimultaneousMoves()
+    {
+        return RangeStreamer.allowSimultaneousMoves && DatabaseDescriptor.getNumTokens() == 1;
+    }
+
     public synchronized void initClient() throws ConfigurationException
     {
         // We don't wait, because we're going to actually try to work on


[4/5] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.3

Posted by ja...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.3


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

Branch: refs/heads/trunk
Commit: d7819232275974f4ee06c62863f64a33cd9a3281
Parents: 501f75b 7aa4dc5
Author: T Jake Luciani <ja...@apache.org>
Authored: Fri Jan 15 09:46:59 2016 -0500
Committer: T Jake Luciani <ja...@apache.org>
Committed: Fri Jan 15 09:46:59 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/service/StorageService.java       | 103 ++++++++++++-------
 2 files changed, 69 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d7819232/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 991077d,a088cfe..950aaaf
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -19,7 -27,16 +19,8 @@@ Merged from 2.2
   * Optimize pending range computation (CASSANDRA-9258)
   * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902)
   * drop/alter user should be case sensitive (CASSANDRA-10817)
 - * jemalloc detection fails due to quoting issues in regexv (CASSANDRA-10946)
 - * (cqlsh) show correct column names for empty result sets (CASSANDRA-9813)
 - * Add new types to Stress (CASSANDRA-9556)
 - * Add property to allow listening on broadcast interface (CASSANDRA-9748)
 - * Fix regression in split size on CqlInputFormat (CASSANDRA-10835)
 - * Better handling of SSL connection errors inter-node (CASSANDRA-10816)
 - * Disable reloading of GossipingPropertyFileSnitch (CASSANDRA-9474)
 - * Verify tables in pseudo-system keyspaces at startup (CASSANDRA-10761)
  Merged from 2.1:
+  * Allow simultaneous bootstrapping with strict consistency when no vnodes are used (CASSANDRA-11005)
   * Log a message when major compaction does not result in a single file (CASSANDRA-10847)
   * (cqlsh) fix cqlsh_copy_tests when vnodes are disabled (CASSANDRA-10997)
   * (cqlsh) Add request timeout option to cqlsh (CASSANDRA-10686)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d7819232/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------


[2/5] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by ja...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/trunk
Commit: e2050c97147505aa28576d17cba3a012bdef31d3
Parents: ee1c145 abe0c67
Author: T Jake Luciani <ja...@apache.org>
Authored: Fri Jan 15 09:44:26 2016 -0500
Committer: T Jake Luciani <ja...@apache.org>
Committed: Fri Jan 15 09:44:26 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/service/StorageService.java       | 56 +++++++++++---------
 2 files changed, 31 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e2050c97/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index d04b9d5,4b87ed0..2bfba80
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,26 -1,8 +1,27 @@@
 -2.1.13
 +2.2.5
 + * Avoid NPE when performing sstable tasks (scrub etc.) (CASSANDRA-10980)
 + * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
 + * Fix error streaming section more than 2GB (CASSANDRA-10961)
 + * (cqlsh) Also apply --connect-timeout to control connection
 +   timeout (CASSANDRA-10959)
 + * Histogram buckets exposed in jmx are sorted incorrectly (CASSANDRA-10975)
 + * Enable GC logging by default (CASSANDRA-10140)
 + * Optimize pending range computation (CASSANDRA-9258)
 + * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902)
 + * drop/alter user should be case sensitive (CASSANDRA-10817)
 + * jemalloc detection fails due to quoting issues in regexv (CASSANDRA-10946)
 + * Support counter-columns for native aggregates (sum,avg,max,min) (CASSANDRA-9977)
 + * (cqlsh) show correct column names for empty result sets (CASSANDRA-9813)
 + * Add new types to Stress (CASSANDRA-9556)
 + * Add property to allow listening on broadcast interface (CASSANDRA-9748)
 + * Fix regression in split size on CqlInputFormat (CASSANDRA-10835)
 + * Better handling of SSL connection errors inter-node (CASSANDRA-10816)
 + * Disable reloading of GossipingPropertyFileSnitch (CASSANDRA-9474)
 + * Verify tables in pseudo-system keyspaces at startup (CASSANDRA-10761)
 +Merged from 2.1:
+  * Allow simultaneous bootstrapping with strict consistency when no vnodes are used (CASSANDRA-11005)
   * Log a message when major compaction does not result in a single file (CASSANDRA-10847)
   * (cqlsh) fix cqlsh_copy_tests when vnodes are disabled (CASSANDRA-10997)
 - * (cqlsh) fix formatting bytearray values (CASSANDRA-10839)
   * (cqlsh) Add request timeout option to cqlsh (CASSANDRA-10686)
   * Avoid AssertionError while submitting hint with LWT (CASSANDRA-10477)
   * If CompactionMetadata is not in stats file, use index summary instead (CASSANDRA-10676)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e2050c97/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index bace082,606670c..34419e9
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -51,16 -51,17 +51,38 @@@ import java.util.concurrent.TimeUnit
  import java.util.concurrent.TimeoutException;
  import java.util.concurrent.atomic.AtomicBoolean;
  import java.util.concurrent.atomic.AtomicInteger;
 -import java.util.concurrent.atomic.AtomicLong;
--
  import javax.management.JMX;
  import javax.management.MBeanServer;
 -import javax.management.Notification;
  import javax.management.NotificationBroadcasterSupport;
  import javax.management.ObjectName;
  import javax.management.openmbean.TabularData;
  import javax.management.openmbean.TabularDataSupport;
  
 -import org.apache.cassandra.auth.Auth;
++import com.google.common.annotations.VisibleForTesting;
++import com.google.common.base.Predicate;
++import com.google.common.collect.ArrayListMultimap;
++import com.google.common.collect.Collections2;
++import com.google.common.collect.HashMultimap;
++import com.google.common.collect.ImmutableSet;
++import com.google.common.collect.Iterables;
++import com.google.common.collect.Lists;
++import com.google.common.collect.Maps;
++import com.google.common.collect.Multimap;
++import com.google.common.collect.Sets;
++import com.google.common.util.concurrent.FutureCallback;
++import com.google.common.util.concurrent.Futures;
++import com.google.common.util.concurrent.ListenableFuture;
++import com.google.common.util.concurrent.Uninterruptibles;
++import org.apache.commons.lang3.StringUtils;
++import org.slf4j.Logger;
++import org.slf4j.LoggerFactory;
++
++import ch.qos.logback.classic.LoggerContext;
++import ch.qos.logback.classic.jmx.JMXConfiguratorMBean;
++import ch.qos.logback.classic.spi.ILoggingEvent;
++import ch.qos.logback.core.Appender;
 +import org.apache.cassandra.auth.AuthKeyspace;
 +import org.apache.cassandra.auth.AuthMigrationListener;
  import org.apache.cassandra.concurrent.ScheduledExecutors;
  import org.apache.cassandra.concurrent.Stage;
  import org.apache.cassandra.concurrent.StageManager;
@@@ -144,35 -142,30 +166,11 @@@ import org.apache.cassandra.utils.FBUti
  import org.apache.cassandra.utils.JVMStabilityInspector;
  import org.apache.cassandra.utils.OutputHandler;
  import org.apache.cassandra.utils.Pair;
 +import org.apache.cassandra.utils.WindowsTimer;
  import org.apache.cassandra.utils.WrappedRunnable;
 -
 -import com.google.common.util.concurrent.*;
 -import org.apache.commons.lang3.StringUtils;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 -
 -import ch.qos.logback.classic.LoggerContext;
 -import ch.qos.logback.classic.jmx.JMXConfiguratorMBean;
 -import ch.qos.logback.classic.spi.ILoggingEvent;
 -import ch.qos.logback.core.Appender;
 -
 -import com.google.common.annotations.VisibleForTesting;
 -import com.google.common.base.Predicate;
 -import com.google.common.collect.ArrayListMultimap;
 -import com.google.common.collect.Collections2;
 -import com.google.common.collect.HashMultimap;
 -import com.google.common.collect.ImmutableSet;
 -import com.google.common.collect.Iterables;
 -import com.google.common.collect.Iterators;
 -import com.google.common.collect.Lists;
 -import com.google.common.collect.Maps;
 -import com.google.common.collect.Multimap;
 -import com.google.common.collect.Sets;
 +import org.apache.cassandra.utils.progress.ProgressEvent;
 +import org.apache.cassandra.utils.progress.ProgressEventType;
 +import org.apache.cassandra.utils.progress.jmx.JMXProgressSupport;
- import org.apache.commons.lang3.StringUtils;
- import org.slf4j.Logger;
- import org.slf4j.LoggerFactory;
- 
- import ch.qos.logback.classic.LoggerContext;
- import ch.qos.logback.classic.jmx.JMXConfiguratorMBean;
- import ch.qos.logback.classic.spi.ILoggingEvent;
- import ch.qos.logback.core.Appender;
- 
- import com.google.common.annotations.VisibleForTesting;
- import com.google.common.base.Predicate;
- import com.google.common.collect.ArrayListMultimap;
- import com.google.common.collect.Collections2;
- import com.google.common.collect.HashMultimap;
- import com.google.common.collect.ImmutableSet;
- import com.google.common.collect.Iterables;
- import com.google.common.collect.Lists;
- import com.google.common.collect.Maps;
- import com.google.common.collect.Multimap;
- import com.google.common.collect.Sets;
- import com.google.common.util.concurrent.FutureCallback;
- import com.google.common.util.concurrent.Futures;
- import com.google.common.util.concurrent.ListenableFuture;
- import com.google.common.util.concurrent.Uninterruptibles;
  
  /**
   * This abstraction contains the token/identifier of this node
@@@ -270,11 -266,10 +268,12 @@@ public class StorageService extends Not
  
      private Collection<Token> bootstrapTokens = null;
  
 -    public void finishBootstrapping()
 -    {
 -        isBootstrapMode = false;
 -    }
 +    // true when keeping strict consistency while bootstrapping
 +    private boolean useStrictConsistency = Boolean.parseBoolean(System.getProperty("cassandra.consistent.rangemovement", "true"));
++    private static final boolean allowSimultaneousMoves = Boolean.valueOf(System.getProperty("cassandra.consistent.simultaneousmoves.allow","false"));
 +    private boolean replacing;
 +
 +    private final StreamStateStore streamStateStore = new StreamStateStore();
  
      /** This method updates the local token on disk  */
      public void setTokens(Collection<Token> tokens)
@@@ -548,7 -545,7 +547,7 @@@
                                                       "Use cassandra.replace_address if you want to replace this node.",
                                                       FBUtilities.getBroadcastAddress()));
          }
-         if (useStrictConsistency)
 -        if (RangeStreamer.useStrictConsistency && !allowSimultaneousMoves())
++        if (useStrictConsistency && !allowSimultaneousMoves())
          {
              for (Map.Entry<InetAddress, EndpointState> entry : Gossiper.instance.getEndpointStates())
              {
@@@ -565,10 -562,47 +564,15 @@@
          Gossiper.instance.resetEndpointStateMap();
      }
  
+     private boolean allowSimultaneousMoves()
+     {
 -        return RangeStreamer.allowSimultaneousMoves && DatabaseDescriptor.getNumTokens() == 1;
 -    }
 -
 -    public synchronized void initClient() throws ConfigurationException
 -    {
 -        // We don't wait, because we're going to actually try to work on
 -        initClient(0);
 -
 -        // sleep a while to allow gossip to warm up (the other nodes need to know about this one before they can reply).
 -        outer:
 -        while (true)
 -        {
 -            Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
 -            for (InetAddress address : Gossiper.instance.getLiveMembers())
 -            {
 -                if (!Gossiper.instance.isFatClient(address))
 -                    break outer;
 -            }
 -        }
 -
 -        // sleep until any schema migrations have finished
 -        while (!MigrationManager.isReadyForBootstrap())
 -        {
 -            Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
 -        }
++        return allowSimultaneousMoves && DatabaseDescriptor.getNumTokens() == 1;
+     }
+ 
 -    public synchronized void initClient(int ringDelay) throws ConfigurationException
 +    // for testing only
 +    public void unsafeInitialize() throws ConfigurationException
      {
 -        if (initialized)
 -        {
 -            if (!isClientMode)
 -                throw new UnsupportedOperationException("StorageService does not support switching modes.");
 -            return;
 -        }
          initialized = true;
 -        isClientMode = true;
 -        logger.info("Starting up client gossip");
 -        setMode(Mode.CLIENT, false);
          Gossiper.instance.register(this);
          Gossiper.instance.start((int) (System.currentTimeMillis() / 1000)); // needed for node-ring gathering.
          Gossiper.instance.addLocalApplicationState(ApplicationState.NET_VERSION, valueFactory.networkVersion());


[5/5] cassandra git commit: Merge branch 'cassandra-3.3' into trunk

Posted by ja...@apache.org.
Merge branch 'cassandra-3.3' into trunk


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

Branch: refs/heads/trunk
Commit: 428806165df6d84969bcdb6caabccbe1b7c87ba0
Parents: 75f7872 d781923
Author: T Jake Luciani <ja...@apache.org>
Authored: Fri Jan 15 09:48:25 2016 -0500
Committer: T Jake Luciani <ja...@apache.org>
Committed: Fri Jan 15 09:48:25 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/service/StorageService.java       | 103 ++++++++++++-------
 2 files changed, 69 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/42880616/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/42880616/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------


[3/5] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ja...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: 7aa4dc5250ede812da73cffa0f715b83d340e2b3
Parents: 5e5ea772 e2050c9
Author: T Jake Luciani <ja...@apache.org>
Authored: Fri Jan 15 09:46:36 2016 -0500
Committer: T Jake Luciani <ja...@apache.org>
Committed: Fri Jan 15 09:46:36 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/service/StorageService.java       | 103 ++++++++++++-------
 2 files changed, 69 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7aa4dc52/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7aa4dc52/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index a501425,34419e9..b3ea8cc
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -37,8 -51,6 +52,7 @@@ import java.util.concurrent.TimeUnit
  import java.util.concurrent.TimeoutException;
  import java.util.concurrent.atomic.AtomicBoolean;
  import java.util.concurrent.atomic.AtomicInteger;
- 
 +import javax.annotation.Nullable;
  import javax.management.JMX;
  import javax.management.MBeanServer;
  import javax.management.NotificationBroadcasterSupport;
@@@ -46,19 -58,56 +60,58 @@@ import javax.management.ObjectName
  import javax.management.openmbean.TabularData;
  import javax.management.openmbean.TabularDataSupport;
  
+ import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Predicate;
+ import com.google.common.collect.ArrayListMultimap;
+ import com.google.common.collect.Collections2;
+ import com.google.common.collect.HashMultimap;
+ import com.google.common.collect.ImmutableSet;
+ import com.google.common.collect.Iterables;
+ import com.google.common.collect.Lists;
+ import com.google.common.collect.Maps;
+ import com.google.common.collect.Multimap;
+ import com.google.common.collect.Sets;
+ import com.google.common.util.concurrent.FutureCallback;
+ import com.google.common.util.concurrent.Futures;
+ import com.google.common.util.concurrent.ListenableFuture;
+ import com.google.common.util.concurrent.Uninterruptibles;
+ import org.apache.commons.lang3.StringUtils;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import ch.qos.logback.classic.LoggerContext;
+ import ch.qos.logback.classic.jmx.JMXConfiguratorMBean;
+ import ch.qos.logback.classic.spi.ILoggingEvent;
+ import ch.qos.logback.core.Appender;
  import org.apache.cassandra.auth.AuthKeyspace;
  import org.apache.cassandra.auth.AuthMigrationListener;
++import org.apache.cassandra.batchlog.BatchRemoveVerbHandler;
++import org.apache.cassandra.batchlog.BatchStoreVerbHandler;
++import org.apache.cassandra.batchlog.BatchlogManager;
  import org.apache.cassandra.concurrent.ScheduledExecutors;
  import org.apache.cassandra.concurrent.Stage;
  import org.apache.cassandra.concurrent.StageManager;
- import org.apache.cassandra.config.*;
- import org.apache.cassandra.db.*;
- import org.apache.cassandra.batchlog.BatchStoreVerbHandler;
- import org.apache.cassandra.batchlog.BatchRemoveVerbHandler;
- import org.apache.cassandra.batchlog.BatchlogManager;
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.config.KSMetaData;
+ import org.apache.cassandra.config.Schema;
 -import org.apache.cassandra.db.BatchlogManager;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.CounterMutationVerbHandler;
+ import org.apache.cassandra.db.DecoratedKey;
+ import org.apache.cassandra.db.DefinitionsUpdateVerbHandler;
 -import org.apache.cassandra.db.HintedHandOffManager;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.db.MigrationRequestVerbHandler;
+ import org.apache.cassandra.db.MutationVerbHandler;
++import org.apache.cassandra.db.RangeSliceVerbHandler;
++import org.apache.cassandra.db.ReadCommandVerbHandler;
+ import org.apache.cassandra.db.ReadRepairVerbHandler;
 -import org.apache.cassandra.db.ReadVerbHandler;
+ import org.apache.cassandra.db.SchemaCheckVerbHandler;
+ import org.apache.cassandra.db.SizeEstimatesRecorder;
+ import org.apache.cassandra.db.SnapshotDetailsTabularData;
+ import org.apache.cassandra.db.SystemKeyspace;
+ import org.apache.cassandra.db.TruncateVerbHandler;
  import org.apache.cassandra.db.commitlog.CommitLog;
  import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
  import org.apache.cassandra.dht.BootStrapper;
  import org.apache.cassandra.dht.IPartitioner;
  import org.apache.cassandra.dht.Range;
@@@ -126,37 -172,6 +179,11 @@@ import org.apache.cassandra.utils.progr
  import org.apache.cassandra.utils.progress.ProgressEventType;
  import org.apache.cassandra.utils.progress.jmx.JMXProgressSupport;
  
- import org.apache.commons.lang3.StringUtils;
- 
- import org.slf4j.Logger;
- import org.slf4j.LoggerFactory;
- import ch.qos.logback.classic.LoggerContext;
- import ch.qos.logback.classic.jmx.JMXConfiguratorMBean;
- import ch.qos.logback.classic.spi.ILoggingEvent;
- import ch.qos.logback.core.Appender;
- 
- import com.google.common.annotations.VisibleForTesting;
- import com.google.common.base.Predicate;
- import com.google.common.collect.ArrayListMultimap;
- import com.google.common.collect.Collections2;
- import com.google.common.collect.HashMultimap;
- import com.google.common.collect.ImmutableSet;
- import com.google.common.collect.Iterables;
- import com.google.common.collect.Lists;
- import com.google.common.collect.Maps;
- import com.google.common.collect.Multimap;
- import com.google.common.collect.Sets;
- import com.google.common.util.concurrent.FutureCallback;
- import com.google.common.util.concurrent.Futures;
- import com.google.common.util.concurrent.ListenableFuture;
- import com.google.common.util.concurrent.Uninterruptibles;
- 
++import static java.util.Arrays.asList;
 +import static java.util.stream.Collectors.toList;
- 
 +import static org.apache.cassandra.index.SecondaryIndexManager.getIndexName;
 +import static org.apache.cassandra.index.SecondaryIndexManager.isIndexColumnFamily;
- import static java.util.Arrays.asList;
 +
  /**
   * This abstraction contains the token/identifier of this node
   * on the identifier space. This token gets gossiped around.