You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2015/09/22 22:13:12 UTC

[1/9] cassandra git commit: Add a separate production debug log for troubleshooting

Repository: cassandra
Updated Branches:
  refs/heads/trunk 56d81ac53 -> a2b1b8abc


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index e57208a..af56c3a 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -984,7 +984,7 @@ public class StorageProxy implements StorageProxyMBean
                 int ttl = HintedHandOffManager.calculateHintTTL(mutation);
                 if (ttl > 0)
                 {
-                    logger.debug("Adding hint for {}", target);
+                    logger.trace("Adding hint for {}", target);
                     writeHintForMutation(mutation, System.currentTimeMillis(), ttl, target);
                     // Notify the handler only for CL == ANY
                     if (responseHandler != null && responseHandler.consistencyLevel == ConsistencyLevel.ANY)
@@ -1390,8 +1390,8 @@ public class StorageProxy implements StorageProxyMBean
                         rows.add(row);
                     }
 
-                    if (logger.isDebugEnabled())
-                        logger.debug("Read: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - exec.handler.start));
+                    if (logger.isTraceEnabled())
+                        logger.trace("Read: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - exec.handler.start));
                 }
                 catch (ReadTimeoutException|ReadFailureException ex)
                 {
@@ -1469,7 +1469,7 @@ public class StorageProxy implements StorageProxyMBean
                         if (Tracing.isTracing())
                             Tracing.trace("Timed out waiting on digest mismatch repair requests");
                         else
-                            logger.debug("Timed out waiting on digest mismatch repair requests");
+                            logger.trace("Timed out waiting on digest mismatch repair requests");
                         // the caught exception here will have CL.ALL from the repair command,
                         // not whatever CL the initial command was at (CASSANDRA-7947)
                         int blockFor = consistencyLevel.blockFor(Keyspace.open(command.getKeyspace()));
@@ -1488,7 +1488,7 @@ public class StorageProxy implements StorageProxyMBean
                         if (Tracing.isTracing())
                             Tracing.trace("Timed out waiting on digest mismatch repair acknowledgements");
                         else
-                            logger.debug("Timed out waiting on digest mismatch repair acknowledgements");
+                            logger.trace("Timed out waiting on digest mismatch repair acknowledgements");
                         int blockFor = consistencyLevel.blockFor(Keyspace.open(command.getKeyspace()));
                         throw new ReadTimeoutException(consistencyLevel, blockFor-1, blockFor, true);
                     }
@@ -1710,7 +1710,7 @@ public class StorageProxy implements StorageProxyMBean
                                   ? 1
                                   : Math.max(1, Math.min(ranges.size(), (int) Math.ceil(command.limit() / resultRowsPerRange)));
 
-                logger.debug("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
+                logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
                              resultRowsPerRange,
                              command.limit(),
                              ranges.size(),
@@ -1896,7 +1896,7 @@ public class StorageProxy implements StorageProxyMBean
                         actualRowsPerRange = fetchedRows / i;
                         concurrencyFactor = Math.max(1, Math.min(ranges.size() - i, Math.round(remainingRows / actualRowsPerRange)));
                     }
-                    logger.debug("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
+                    logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
                                  actualRowsPerRange, (int) remainingRows, concurrencyFactor);
                 }
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 64662bf..1abc928 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -295,7 +295,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("get_slice");
+            logger.trace("get_slice");
         }
 
         try
@@ -343,7 +343,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("multiget_slice");
+            logger.trace("multiget_slice");
         }
 
         try
@@ -450,7 +450,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("get");
+            logger.trace("get");
         }
 
         try
@@ -519,7 +519,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("get_count");
+            logger.trace("get_count");
         }
 
         try
@@ -541,7 +541,7 @@ public class CassandraServer implements Cassandra.Iface
                 int averageColumnSize = (int) (cfs.metric.meanRowSize.getValue() / cfs.getMeanColumns());
                 pageSize = Math.min(COUNT_PAGE_SIZE, 4 * 1024 * 1024 / averageColumnSize);
                 pageSize = Math.max(2, pageSize);
-                logger.debug("average row column size is {}; using pageSize of {}", averageColumnSize, pageSize);
+                logger.trace("average row column size is {}; using pageSize of {}", averageColumnSize, pageSize);
             }
             else
             {
@@ -599,7 +599,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("multiget_count");
+            logger.trace("multiget_count");
         }
 
         try
@@ -680,7 +680,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("insert");
+            logger.trace("insert");
         }
 
         try
@@ -720,7 +720,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("cas");
+            logger.trace("cas");
         }
 
         try
@@ -947,7 +947,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("batch_mutate");
+            logger.trace("batch_mutate");
         }
 
         try
@@ -980,7 +980,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("atomic_batch_mutate");
+            logger.trace("atomic_batch_mutate");
         }
 
         try
@@ -1039,7 +1039,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("remove");
+            logger.trace("remove");
         }
 
         try
@@ -1126,7 +1126,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("range_slice");
+            logger.trace("range_slice");
         }
 
         try
@@ -1210,7 +1210,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("get_paged_slice");
+            logger.trace("get_paged_slice");
         }
 
         try
@@ -1305,7 +1305,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("scan");
+            logger.trace("scan");
         }
 
         try
@@ -1495,7 +1495,7 @@ public class CassandraServer implements Cassandra.Iface
     public String system_add_column_family(CfDef cf_def)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("add_column_family");
+        logger.trace("add_column_family");
 
         try
         {
@@ -1522,7 +1522,7 @@ public class CassandraServer implements Cassandra.Iface
     public String system_drop_column_family(String column_family)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("drop_column_family");
+        logger.trace("drop_column_family");
 
         ThriftClientState cState = state();
 
@@ -1542,7 +1542,7 @@ public class CassandraServer implements Cassandra.Iface
     public String system_add_keyspace(KsDef ks_def)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("add_keyspace");
+        logger.trace("add_keyspace");
 
         try
         {
@@ -1583,7 +1583,7 @@ public class CassandraServer implements Cassandra.Iface
     public String system_drop_keyspace(String keyspace)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("drop_keyspace");
+        logger.trace("drop_keyspace");
 
         try
         {
@@ -1605,7 +1605,7 @@ public class CassandraServer implements Cassandra.Iface
     public String system_update_keyspace(KsDef ks_def)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("update_keyspace");
+        logger.trace("update_keyspace");
 
         try
         {
@@ -1627,7 +1627,7 @@ public class CassandraServer implements Cassandra.Iface
     public String system_update_column_family(CfDef cf_def)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("update_column_family");
+        logger.trace("update_column_family");
 
         try
         {
@@ -1674,7 +1674,7 @@ public class CassandraServer implements Cassandra.Iface
             }
             else
             {
-                logger.debug("truncating {}.{}", cState.getKeyspace(), cfname);
+                logger.trace("truncating {}.{}", cState.getKeyspace(), cfname);
             }
 
             schedule(DatabaseDescriptor.getTruncateRpcTimeout());
@@ -1723,7 +1723,7 @@ public class CassandraServer implements Cassandra.Iface
 
     public Map<String, List<String>> describe_schema_versions() throws TException, InvalidRequestException
     {
-        logger.debug("checking schema agreement");
+        logger.trace("checking schema agreement");
         return StorageProxy.describeSchemaVersions();
     }
 
@@ -1741,7 +1741,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("add");
+            logger.trace("add");
         }
 
         try
@@ -1797,7 +1797,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("remove_counter");
+            logger.trace("remove_counter");
         }
 
         try
@@ -1890,7 +1890,7 @@ public class CassandraServer implements Cassandra.Iface
             }
             else
             {
-                logger.debug("execute_cql3_query");
+                logger.trace("execute_cql3_query");
             }
 
             ThriftClientState cState = state();
@@ -1920,7 +1920,7 @@ public class CassandraServer implements Cassandra.Iface
 
     public CqlPreparedResult prepare_cql3_query(ByteBuffer query, Compression compression) throws TException
     {
-        logger.debug("prepare_cql3_query");
+        logger.trace("prepare_cql3_query");
 
         String queryString = uncompress(query, compression);
         ThriftClientState cState = state();
@@ -1952,7 +1952,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("execute_prepared_cql3_query");
+            logger.trace("execute_prepared_cql3_query");
         }
 
         try
@@ -2001,7 +2001,7 @@ public class CassandraServer implements Cassandra.Iface
         }
         else
         {
-            logger.debug("get_multi_slice");
+            logger.trace("get_multi_slice");
         }
         try 
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index a025004..bde5310 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -126,7 +126,7 @@ public class CustomTThreadPoolServer extends TServer
             catch (RejectedExecutionException e)
             {
                 // worker thread decremented activeClients but hadn't finished exiting
-                logger.debug("Dropping client connection because our limit of {} has been reached", args.maxWorkerThreads);
+                logger.trace("Dropping client connection because our limit of {} has been reached", args.maxWorkerThreads);
                 continue;
             }
 
@@ -211,7 +211,7 @@ public class CustomTThreadPoolServer extends TServer
             {
                 // Assume the client died and continue silently
                 // Log at debug to allow debugging of "frame too large" errors (see CASSANDRA-3142).
-                logger.debug("Thrift transport error occurred during processing of message.", ttx);
+                logger.trace("Thrift transport error occurred during processing of message.", ttx);
             }
             catch (TException tx)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index 3768952..d735676 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -451,8 +451,8 @@ public class ThriftValidation
         }
         catch (MarshalException me)
         {
-            if (logger.isDebugEnabled())
-                logger.debug("rejecting invalid value {}", ByteBufferUtil.bytesToHex(summarize(column.value)));
+            if (logger.isTraceEnabled())
+                logger.trace("rejecting invalid value {}", ByteBufferUtil.bytesToHex(summarize(column.value)));
 
             throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("(%s) [%s][%s][%s] failed validation",
                                                                       me.getMessage(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index 3f9f54d..ccc2637 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -152,7 +152,7 @@ public class Tracing
         TraceState state = this.state.get();
         if (state == null) // inline isTracing to avoid implicit two calls to state.get()
         {
-            logger.debug("request complete");
+            logger.trace("request complete");
         }
         else
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/transport/Message.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index 440d481..ab794df 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -503,7 +503,7 @@ public abstract class Message
 
                 QueryState qstate = connection.validateNewMessage(request.type, connection.getVersion(), request.getStreamId());
 
-                logger.debug("Received: {}, v={}", request, connection.getVersion());
+                logger.trace("Received: {}, v={}", request, connection.getVersion());
                 response = request.execute(qstate);
                 response.setStreamId(request.getStreamId());
                 response.setWarnings(ClientWarn.getWarnings());
@@ -522,7 +522,7 @@ public abstract class Message
                 ClientWarn.resetWarnings();
             }
 
-            logger.debug("Responding: {}, v={}", response, connection.getVersion());
+            logger.trace("Responding: {}, v={}", response, connection.getVersion());
             flush(new FlushItem(ctx, response, request.getSourceFrame()));
         }
 
@@ -597,7 +597,7 @@ public abstract class Message
                 if (ioExceptionsAtDebugLevel.contains(exception.getMessage()))
                 {
                     // Likely unclean client disconnects
-                    logger.debug(message, exception);
+                    logger.trace(message, exception);
                 }
                 else
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/triggers/CustomClassLoader.java b/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
index ecb34a2..965da4b 100644
--- a/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
+++ b/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
@@ -112,7 +112,7 @@ public class CustomClassLoader extends URLClassLoader
         }
         catch (ClassNotFoundException ex)
         {
-            logger.debug("Class not found using parent class loader,", ex);
+            logger.trace("Class not found using parent class loader,", ex);
             // Don't throw the exception here, try triggers directory.
         }
         Class<?> clazz = this.findClass(name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/utils/CLibrary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CLibrary.java b/src/java/org/apache/cassandra/utils/CLibrary.java
index fed314b..4ca5d53 100644
--- a/src/java/org/apache/cassandra/utils/CLibrary.java
+++ b/src/java/org/apache/cassandra/utils/CLibrary.java
@@ -70,7 +70,7 @@ public final class CLibrary
         catch (UnsatisfiedLinkError e)
         {
             logger.warn("JNA link failure, one or more native method will be unavailable.");
-            logger.debug("JNA link failure details: {}", e.getMessage());
+            logger.trace("JNA link failure details: {}", e.getMessage());
         }
         catch (NoSuchMethodError e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/EstimatedHistogram.java b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
index a5c51c8..93d142a 100644
--- a/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
+++ b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
@@ -234,7 +234,7 @@ public class EstimatedHistogram
     }
 
     /**
-     * log.debug() every record in the histogram
+     * log.trace() every record in the histogram
      *
      * @param log
      */
@@ -265,7 +265,7 @@ public class EstimatedHistogram
             // calculation, and accept the unnecessary whitespace prefixes that will occasionally occur
             if (i == 0 && count == 0)
                 continue;
-            log.debug(String.format(formatstr, names[i], count));
+            log.trace(String.format(formatstr, names[i], count));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/utils/Mx4jTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Mx4jTool.java b/src/java/org/apache/cassandra/utils/Mx4jTool.java
index e8fdb29..41a5b33 100644
--- a/src/java/org/apache/cassandra/utils/Mx4jTool.java
+++ b/src/java/org/apache/cassandra/utils/Mx4jTool.java
@@ -42,7 +42,7 @@ public class Mx4jTool
     {
         try
         {
-            logger.debug("Will try to load mx4j now, if it's in the classpath");
+            logger.trace("Will try to load mx4j now, if it's in the classpath");
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             ObjectName processorName = new ObjectName("Server:name=XSLTProcessor");
 
@@ -65,7 +65,7 @@ public class Mx4jTool
         }
         catch (ClassNotFoundException e)
         {
-            logger.debug("Will not load MX4J, mx4j-tools.jar is not in the classpath");
+            logger.trace("Will not load MX4J, mx4j-tools.jar is not in the classpath");
         }
         catch(Exception e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/utils/OutputHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/OutputHandler.java b/src/java/org/apache/cassandra/utils/OutputHandler.java
index b203663..88c54ce 100644
--- a/src/java/org/apache/cassandra/utils/OutputHandler.java
+++ b/src/java/org/apache/cassandra/utils/OutputHandler.java
@@ -44,7 +44,7 @@ public interface OutputHandler
 
         public void debug(String msg)
         {
-            logger.debug(msg);
+            logger.trace(msg);
         }
 
         public void warn(String msg)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/utils/TopKSampler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/TopKSampler.java b/src/java/org/apache/cassandra/utils/TopKSampler.java
index a8bd602..37fcb60 100644
--- a/src/java/org/apache/cassandra/utils/TopKSampler.java
+++ b/src/java/org/apache/cassandra/utils/TopKSampler.java
@@ -110,7 +110,7 @@ public class TopKSampler<T>
                                 hll.offerHashed(hash);
                             } catch (Exception e)
                             {
-                                logger.debug("Failure to offer sample", e);
+                                logger.trace("Failure to offer sample", e);
                             }
                         }
                     }


[2/9] cassandra git commit: Add a separate production debug log for troubleshooting

Posted by bl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index 9a88164..ddbc3a1 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -71,7 +71,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
         maxSSTableSizeInMB = configuredMaxSSTableSize;
 
         manifest = new LeveledManifest(cfs, this.maxSSTableSizeInMB, localOptions);
-        logger.debug("Created {}", manifest);
+        logger.trace("Created {}", manifest);
     }
 
     public int getLevelSize(int i)
@@ -101,7 +101,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
                 SSTableReader sstable = findDroppableSSTable(gcBefore);
                 if (sstable == null)
                 {
-                    logger.debug("No compaction necessary for {}", this);
+                    logger.trace("No compaction necessary for {}", this);
                     return null;
                 }
                 candidate = new LeveledManifest.CompactionCandidate(Collections.singleton(sstable),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index 0d0928f..d90318f 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -115,7 +115,7 @@ public class LeveledManifest
         if (canAddSSTable(reader))
         {
             // adding the sstable does not cause overlap in the level
-            logger.debug("Adding {} to L{}", reader, level);
+            logger.trace("Adding {} to L{}", reader, level);
             generations[level].add(reader);
         }
         else
@@ -146,8 +146,8 @@ public class LeveledManifest
     {
         assert !removed.isEmpty(); // use add() instead of promote when adding new sstables
         logDistribution();
-        if (logger.isDebugEnabled())
-            logger.debug("Replacing [{}]", toString(removed));
+        if (logger.isTraceEnabled())
+            logger.trace("Replacing [{}]", toString(removed));
 
         // the level for the added sstables is the max of the removed ones,
         // plus one if the removed were all on the same level
@@ -163,8 +163,8 @@ public class LeveledManifest
         if (added.isEmpty())
             return;
 
-        if (logger.isDebugEnabled())
-            logger.debug("Adding [{}]", toString(added));
+        if (logger.isTraceEnabled())
+            logger.trace("Adding [{}]", toString(added));
 
         for (SSTableReader ssTableReader : added)
             add(ssTableReader);
@@ -317,7 +317,7 @@ public class LeveledManifest
             Set<SSTableReader> sstablesInLevel = Sets.newHashSet(sstables);
             Set<SSTableReader> remaining = Sets.difference(sstablesInLevel, cfs.getTracker().getCompacting());
             double score = (double) SSTableReader.getTotalBytes(remaining) / (double)maxBytesForLevel(i, maxSSTableSizeInBytes);
-            logger.debug("Compaction score for level {} is {}", i, score);
+            logger.trace("Compaction score for level {} is {}", i, score);
 
             if (score > 1.001)
             {
@@ -327,7 +327,7 @@ public class LeveledManifest
                     List<SSTableReader> mostInteresting = getSSTablesForSTCS(getLevel(0));
                     if (!mostInteresting.isEmpty())
                     {
-                        logger.debug("L0 is too far behind, performing size-tiering there first");
+                        logger.trace("L0 is too far behind, performing size-tiering there first");
                         return new CompactionCandidate(mostInteresting, 0, Long.MAX_VALUE);
                     }
                 }
@@ -338,13 +338,13 @@ public class LeveledManifest
                 {
                     int nextLevel = getNextLevel(candidates);
                     candidates = getOverlappingStarvedSSTables(nextLevel, candidates);
-                    if (logger.isDebugEnabled())
-                        logger.debug("Compaction candidates for L{} are {}", i, toString(candidates));
+                    if (logger.isTraceEnabled())
+                        logger.trace("Compaction candidates for L{} are {}", i, toString(candidates));
                     return new CompactionCandidate(candidates, nextLevel, cfs.getCompactionStrategy().getMaxSSTableBytes());
                 }
                 else
                 {
-                    logger.debug("No compaction candidates for L{}", i);
+                    logger.trace("No compaction candidates for L{}", i);
                 }
             }
         }
@@ -387,10 +387,10 @@ public class LeveledManifest
         for (int i = generations.length - 1; i > 0; i--)
             compactionCounter[i]++;
         compactionCounter[targetLevel] = 0;
-        if (logger.isDebugEnabled())
+        if (logger.isTraceEnabled())
         {
             for (int j = 0; j < compactionCounter.length; j++)
-                logger.debug("CompactionCounter: {}: {}", j, compactionCounter[j]);
+                logger.trace("CompactionCounter: {}: {}", j, compactionCounter[j]);
         }
 
         for (int i = generations.length - 1; i > 0; i--)
@@ -451,13 +451,13 @@ public class LeveledManifest
 
     private void logDistribution()
     {
-        if (logger.isDebugEnabled())
+        if (logger.isTraceEnabled())
         {
             for (int i = 0; i < generations.length; i++)
             {
                 if (!getLevel(i).isEmpty())
                 {
-                    logger.debug("L{} contains {} SSTables ({} bytes) in {}",
+                    logger.trace("L{} contains {} SSTables ({} bytes) in {}",
                                  i, getLevel(i).size(), SSTableReader.getTotalBytes(getLevel(i)), this);
                 }
             }
@@ -539,7 +539,7 @@ public class LeveledManifest
     private Collection<SSTableReader> getCandidatesFor(int level)
     {
         assert !getLevel(level).isEmpty();
-        logger.debug("Choosing candidates for L{}", level);
+        logger.trace("Choosing candidates for L{}", level);
 
         final Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
 
@@ -703,7 +703,7 @@ public class LeveledManifest
             tasks += estimated[i];
         }
 
-        logger.debug("Estimating {} compactions to do for {}.{}",
+        logger.trace("Estimating {} compactions to do for {}.{}",
                      Arrays.toString(estimated), cfs.keyspace.getName(), cfs.name);
         return Ints.checkedCast(tasks);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index 4ba2378..b4125bb 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -82,7 +82,7 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
         Iterable<SSTableReader> candidates = filterSuspectSSTables(Sets.intersection(cfs.getUncompactingSSTables(), sstables));
 
         List<List<SSTableReader>> buckets = getBuckets(createSSTableAndLengthPairs(candidates), sizeTieredOptions.bucketHigh, sizeTieredOptions.bucketLow, sizeTieredOptions.minSSTableSize);
-        logger.debug("Compaction buckets are {}", buckets);
+        logger.trace("Compaction buckets are {}", buckets);
         updateEstimatedCompactionsByTasks(buckets);
         List<SSTableReader> mostInteresting = mostInterestingBucket(buckets, minThreshold, maxThreshold);
         if (!mostInteresting.isEmpty())
@@ -210,7 +210,7 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
         LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
         if (transaction == null)
         {
-            logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
+            logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
index 1e73daf..9daa0c5 100644
--- a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
@@ -63,7 +63,7 @@ public final class WrappingCompactionStrategy extends AbstractCompactionStrategy
         super(cfs, cfs.metadata.compactionStrategyOptions);
         reloadCompactionStrategy(cfs.metadata);
         cfs.getTracker().subscribe(this);
-        logger.debug("{} subscribed to the data tracker.", this);
+        logger.trace("{} subscribed to the data tracker.", this);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
index 7d88458..6611b3f 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -45,7 +45,7 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
     public DefaultCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
     {
         super(cfs, txn, nonExpiredSSTables, offline);
-        logger.debug("Expected bloom filter size : {}", estimatedTotalKeys);
+        logger.trace("Expected bloom filter size : {}", estimatedTotalKeys);
         long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
         File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
         @SuppressWarnings("resource")

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
index 9ff1325..ed07df9 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -93,7 +93,7 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
                                                                             new MetadataCollector(allSSTables, cfs.metadata.comparator, 0));
 
         sstableWriter.switchWriter(writer);
-        logger.debug("Ratios={}, expectedKeys = {}, totalSize = {}, currentPartitionsToWrite = {}, currentBytesToWrite = {}", ratios, estimatedTotalKeys, totalSize, currentPartitionsToWrite, currentBytesToWrite);
+        logger.trace("Ratios={}, expectedKeys = {}, totalSize = {}, currentPartitionsToWrite = {}, currentBytesToWrite = {}", ratios, estimatedTotalKeys, totalSize, currentPartitionsToWrite, currentBytesToWrite);
     }
 
     @Override
@@ -114,7 +114,7 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
                                                                                 cfs.partitioner,
                                                                                 new MetadataCollector(allSSTables, cfs.metadata.comparator, 0));
             sstableWriter.switchWriter(writer);
-            logger.debug("Switching writer, currentPartitionsToWrite = {}", currentPartitionsToWrite);
+            logger.trace("Switching writer, currentPartitionsToWrite = {}", currentPartitionsToWrite);
         }
         return rie != null;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
index 50ab57d..c6d93fe 100644
--- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
@@ -261,7 +261,7 @@ public abstract class ExtendedFilter
             {
                 if (data.getColumn(data.getComparator().cellFromByteBuffer(expr.column)) == null)
                 {
-                    logger.debug("adding extraFilter to cover additional expressions");
+                    logger.trace("adding extraFilter to cover additional expressions");
                     return true;
                 }
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
index ba48350..854d688 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -101,8 +101,8 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata, false, 1);
         cfi.addTombstone(makeIndexColumnName(rowKey, cell), localDeletionTime, cell.timestamp());
         indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
-        if (logger.isDebugEnabled())
-            logger.debug("removed index entry for cleaned-up value {}:{}", valueKey, cfi);
+        if (logger.isTraceEnabled())
+            logger.trace("removed index entry for cleaned-up value {}:{}", valueKey, cfi);
     }
 
     public void insert(ByteBuffer rowKey, Cell cell, OpOrder.Group opGroup)
@@ -119,8 +119,8 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         {
             cfi.addColumn(new BufferCell(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, cell.timestamp()));
         }
-        if (logger.isDebugEnabled())
-            logger.debug("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.getKey()), cfi);
+        if (logger.isTraceEnabled())
+            logger.trace("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.getKey()), cfi);
 
         indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
index e88d456..0be78cc 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
@@ -142,8 +142,8 @@ public abstract class CompositesIndex extends AbstractSimplePerColumnSecondaryIn
         ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata);
         cfi.addTombstone(entry.indexEntry, localDeletionTime, entry.timestamp);
         indexCfs.apply(entry.indexValue, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
-        if (logger.isDebugEnabled())
-            logger.debug("removed index entry for cleaned-up value {}:{}", entry.indexValue, cfi);
+        if (logger.isTraceEnabled())
+            logger.trace("removed index entry for cleaned-up value {}:{}", entry.indexValue, cfi);
     }
 
     protected AbstractType<?> getExpressionComparator()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
index 88453df..2f85e35 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
@@ -98,8 +98,8 @@ public class CompositesSearcher extends SecondaryIndexSearcher
         assert index.getIndexCfs() != null;
         final DecoratedKey indexKey = index.getIndexKeyFor(primary.value);
 
-        if (logger.isDebugEnabled())
-            logger.debug("Most-selective indexed predicate is {}", index.expressionString(primary));
+        if (logger.isTraceEnabled())
+            logger.trace("Most-selective indexed predicate is {}", index.expressionString(primary));
 
         /*
          * XXX: If the range requested is a token range, we'll have to start at the beginning (and stop at the end) of
@@ -240,7 +240,7 @@ public class CompositesSearcher extends SecondaryIndexSearcher
                             }
                             else
                             {
-                                logger.debug("Skipping entry {} before assigned scan range", dk.getToken());
+                                logger.trace("Skipping entry {} before assigned scan range", dk.getToken());
                                 continue;
                             }
                         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
index b4fd0ba..2f0420f 100644
--- a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
@@ -73,8 +73,8 @@ public class KeysSearcher extends SecondaryIndexSearcher
         assert index.getIndexCfs() != null;
         final DecoratedKey indexKey = index.getIndexKeyFor(primary.value);
 
-        if (logger.isDebugEnabled())
-            logger.debug("Most-selective indexed predicate is {}",
+        if (logger.isTraceEnabled())
+            logger.trace("Most-selective indexed predicate is {}",
                          ((AbstractSimplePerColumnSecondaryIndex) index).expressionString(primary));
 
         /*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
index dfa30c7..9b52269 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -150,7 +150,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
     {
         assert staged.isEmpty() : "must be no actions introduced between prepareToCommit and a commit";
 
-        logger.debug("Committing update:{}, obsolete:{}", staged.update, staged.obsolete);
+        logger.trace("Committing update:{}, obsolete:{}", staged.update, staged.obsolete);
 
         // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done
         // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size
@@ -167,15 +167,15 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
      */
     public Throwable doAbort(Throwable accumulate)
     {
-        if (logger.isDebugEnabled())
-            logger.debug("Aborting transaction over {}, with ({},{}) logged and ({},{}) staged", originals, logged.update, logged.obsolete, staged.update, staged.obsolete);
+        if (logger.isTraceEnabled())
+            logger.trace("Aborting transaction over {}, with ({},{}) logged and ({},{}) staged", originals, logged.update, logged.obsolete, staged.update, staged.obsolete);
 
         if (logged.isEmpty() && staged.isEmpty())
             return accumulate;
 
         // mark obsolete all readers that are not versions of those present in the original set
         Iterable<SSTableReader> obsolete = filterOut(concatUniq(staged.update, logged.update), originals);
-        logger.debug("Obsoleting {}", obsolete);
+        logger.trace("Obsoleting {}", obsolete);
         // we don't pass the tracker in for the obsoletion, since these readers have never been notified externally
         // nor had their size accounting affected
         accumulate = markObsolete(null, obsolete, accumulate);
@@ -221,8 +221,8 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
     }
     private Throwable checkpoint(Throwable accumulate)
     {
-        if (logger.isDebugEnabled())
-            logger.debug("Checkpointing update:{}, obsolete:{}", staged.update, staged.obsolete);
+        if (logger.isTraceEnabled())
+            logger.trace("Checkpointing update:{}, obsolete:{}", staged.update, staged.obsolete);
 
         if (staged.isEmpty())
             return accumulate;
@@ -275,7 +275,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
      */
     public void obsolete(SSTableReader reader)
     {
-        logger.debug("Staging for obsolescence {}", reader);
+        logger.trace("Staging for obsolescence {}", reader);
         // check this is: a reader guarded by the transaction, an instance we have already worked with
         // and that we haven't already obsoleted it, nor do we have other changes staged for it
         assert identities.contains(reader.instanceId) : "only reader instances that have previously been provided may be obsoleted: " + reader;
@@ -291,7 +291,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
      */
     public void obsoleteOriginals()
     {
-        logger.debug("Staging for obsolescence {}", originals);
+        logger.trace("Staging for obsolescence {}", originals);
         // if we're obsoleting, we should have no staged updates for the original files
         assert Iterables.isEmpty(filterIn(staged.update, originals)) : staged.update;
 
@@ -381,7 +381,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
      */
     public void cancel(SSTableReader cancel)
     {
-        logger.debug("Cancelling {} from transaction", cancel);
+        logger.trace("Cancelling {} from transaction", cancel);
         assert originals.contains(cancel) : "may only cancel a reader in the 'original' set: " + cancel + " vs " + originals;
         assert !(staged.contains(cancel) || logged.contains(cancel)) : "may only cancel a reader that has not been updated or obsoleted in this transaction: " + cancel;
         originals.remove(cancel);
@@ -405,7 +405,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
      */
     public LifecycleTransaction split(Collection<SSTableReader> readers)
     {
-        logger.debug("Splitting {} into new transaction", readers);
+        logger.trace("Splitting {} into new transaction", readers);
         checkUnused();
         for (SSTableReader reader : readers)
             assert identities.contains(reader.instanceId) : "may only split the same reader instance the transaction was opened with: " + reader;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index f1c4685..a074216 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -137,8 +137,8 @@ public class Tracker
         long add = 0;
         for (SSTableReader sstable : newSSTables)
         {
-            if (logger.isDebugEnabled())
-                logger.debug("adding {} to list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
+            if (logger.isTraceEnabled())
+                logger.trace("adding {} to list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
             try
             {
                 add += sstable.bytesOnDisk();
@@ -151,8 +151,8 @@ public class Tracker
         long subtract = 0;
         for (SSTableReader sstable : oldSSTables)
         {
-            if (logger.isDebugEnabled())
-                logger.debug("removing {} from list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
+            if (logger.isTraceEnabled())
+                logger.trace("removing {} from list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
             try
             {
                 subtract += sstable.bytesOnDisk();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index aca05f0..a6b1ad7 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -65,7 +65,7 @@ public class BootStrapper extends ProgressEventNotifierSupport
 
     public ListenableFuture<StreamState> bootstrap(StreamStateStore stateStore, boolean useStrictConsistency)
     {
-        logger.debug("Beginning bootstrap process");
+        logger.trace("Beginning bootstrap process");
 
         RangeStreamer streamer = new RangeStreamer(tokenMetadata,
                                                    tokens,
@@ -159,7 +159,7 @@ public class BootStrapper extends ProgressEventNotifierSupport
         // if user specified tokens, use those
         if (initialTokens.size() > 0)
         {
-            logger.debug("tokens manually specified as {}",  initialTokens);
+            logger.trace("tokens manually specified as {}",  initialTokens);
             List<Token> tokens = new ArrayList<>(initialTokens.size());
             for (String tokenString : initialTokens)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/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 68c8a11..8f2dc12 100644
--- a/src/java/org/apache/cassandra/dht/RangeStreamer.java
+++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java
@@ -146,18 +146,18 @@ public class RangeStreamer
         Multimap<Range<Token>, InetAddress> rangesForKeyspace = useStrictSourcesForRanges(keyspaceName)
                 ? getAllRangesWithStrictSourcesFor(keyspaceName, ranges) : getAllRangesWithSourcesFor(keyspaceName, ranges);
 
-        if (logger.isDebugEnabled())
+        if (logger.isTraceEnabled())
         {
             for (Map.Entry<Range<Token>, InetAddress> entry : rangesForKeyspace.entries())
-                logger.debug(String.format("%s: range %s exists on %s", description, entry.getKey(), entry.getValue()));
+                logger.trace(String.format("%s: range %s exists on %s", description, entry.getKey(), entry.getValue()));
         }
 
         for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : getRangeFetchMap(rangesForKeyspace, sourceFilters, keyspaceName).asMap().entrySet())
         {
-            if (logger.isDebugEnabled())
+            if (logger.isTraceEnabled())
             {
                 for (Range<Token> r : entry.getValue())
-                    logger.debug(String.format("%s: range %s from source %s for keyspace %s", description, r, entry.getKey(), keyspaceName));
+                    logger.trace(String.format("%s: range %s from source %s for keyspace %s", description, r, entry.getKey(), keyspaceName));
             }
             toFetch.put(keyspaceName, entry);
         }
@@ -339,8 +339,8 @@ public class RangeStreamer
                 logger.info("Some ranges of {} are already available. Skipping streaming those ranges.", availableRanges);
             }
 
-            if (logger.isDebugEnabled())
-                logger.debug("{}ing from {} ranges {}", description, source, StringUtils.join(ranges, ", "));
+            if (logger.isTraceEnabled())
+                logger.trace("{}ing from {} ranges {}", description, source, StringUtils.join(ranges, ", "));
             /* Send messages to respective folks to stream data over to me */
             streamPlan.requestRanges(source, preferred, keyspace, ranges);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
index 9c45bfe..103d478 100644
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -81,7 +81,7 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
         keyspace = ConfigHelper.getInputKeyspace(conf);
         cfName = ConfigHelper.getInputColumnFamily(conf);
         partitioner = ConfigHelper.getInputPartitioner(conf);
-        logger.debug("partitioner is {}", partitioner);
+        logger.trace("partitioner is {}", partitioner);
 
         // canonical ranges and nodes holding replicas
         Map<TokenRange, Set<Host>> masterRangeNodes = getRangeMap(conf, keyspace);
@@ -219,7 +219,7 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
                                     subSplits.get(subSplit),
                                     endpoints);
 
-                    logger.debug("adding {}", split);
+                    logger.trace("adding {}", split);
                     splits.add(split);
                 }
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
index 4662fa5..87cb791 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
@@ -63,7 +63,7 @@ public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<Byt
     @SuppressWarnings("resource")
     public static Cassandra.Client createAuthenticatedClient(String location, int port, Configuration conf) throws Exception
     {
-        logger.debug("Creating authenticated client for CF input format");
+        logger.trace("Creating authenticated client for CF input format");
         TTransport transport;
         try
         {
@@ -86,7 +86,7 @@ public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<Byt
             AuthenticationRequest authRequest = new AuthenticationRequest(creds);
             client.login(authRequest);
         }
-        logger.debug("Authenticated client for CF input format created successfully");
+        logger.trace("Authenticated client for CF input format created successfully");
         return client;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
index 92e3829..edc988b 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
@@ -117,7 +117,7 @@ public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutat
     @SuppressWarnings("resource")
     public static Cassandra.Client createAuthenticatedClient(String host, int port, Configuration conf) throws Exception
     {
-        logger.debug("Creating authenticated client for CF output format");
+        logger.trace("Creating authenticated client for CF output format");
         TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(host, port);
         TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
         Cassandra.Client client = new Cassandra.Client(binaryProtocol);
@@ -127,7 +127,7 @@ public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutat
         if ((user != null) && (password != null))
             login(user, password, client);
 
-        logger.debug("Authenticated client for CF output format created successfully");
+        logger.trace("Authenticated client for CF output format created successfully");
         return client;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index aee730d..97dc497 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -170,14 +170,14 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
         }
 
         iter = widerows ? new WideRowIterator() : new StaticRowIterator();
-        logger.debug("created {}", iter);
+        logger.trace("created {}", iter);
     }
 
     public boolean nextKeyValue() throws IOException
     {
         if (!iter.hasNext())
         {
-            logger.debug("Finished scanning {} rows (estimate was: {})", iter.rowsRead(), totalRowCount);
+            logger.trace("Finished scanning {} rows (estimate was: {})", iter.rowsRead(), totalRowCount);
             return false;
         }
 
@@ -443,7 +443,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
             else
             {
                 KeySlice lastRow = Iterables.getLast(rows);
-                logger.debug("Starting with last-seen row {}", lastRow.key);
+                logger.trace("Starting with last-seen row {}", lastRow.key);
                 keyRange = new KeyRange(batchSize)
                           .setStart_key(lastRow.key)
                           .setEnd_token(split.getEndToken())
@@ -456,7 +456,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
                 int n = 0;
                 for (KeySlice row : rows)
                     n += row.columns.size();
-                logger.debug("read {} columns in {} rows for {} starting with {}",
+                logger.trace("read {} columns in {} rows for {} starting with {}",
                              new Object[]{ n, rows.size(), keyRange, lastColumn });
 
                 wideColumns = Iterators.peekingIterator(new WideColumnIterator(rows));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
index 6db851d..b3e440d 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
@@ -156,10 +156,10 @@ public class CqlRecordReader extends RecordReader<Long, Row>
 
         if (StringUtils.isEmpty(cqlQuery))
             cqlQuery = buildQuery();
-        logger.debug("cqlQuery {}", cqlQuery);
+        logger.trace("cqlQuery {}", cqlQuery);
 
         rowIterator = new RowIterator();
-        logger.debug("created {}", rowIterator);
+        logger.trace("created {}", rowIterator);
     }
 
     public void close()
@@ -194,7 +194,7 @@ public class CqlRecordReader extends RecordReader<Long, Row>
     {
         if (!rowIterator.hasNext())
         {
-            logger.debug("Finished scanning {} rows (estimate was: {})", rowIterator.totalRead, totalRowCount);
+            logger.trace("Finished scanning {} rows (estimate was: {})", rowIterator.totalRead, totalRowCount);
             return false;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java b/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java
index da0822a..5c8d3c5 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java
@@ -71,7 +71,7 @@ class LimitedLocalNodeFirstLocalBalancingPolicy implements LoadBalancingPolicy
                 logger.warn("Invalid replica host name: {}, skipping it", replica);
             }
         }
-        logger.debug("Created instance with the following replicas: {}", Arrays.asList(replicas));
+        logger.trace("Created instance with the following replicas: {}", Arrays.asList(replicas));
     }
 
     @Override
@@ -86,7 +86,7 @@ class LimitedLocalNodeFirstLocalBalancingPolicy implements LoadBalancingPolicy
             }
         }
         liveReplicaHosts.addAll(replicaHosts);
-        logger.debug("Initialized with replica hosts: {}", replicaHosts);
+        logger.trace("Initialized with replica hosts: {}", replicaHosts);
     }
 
     @Override
@@ -127,7 +127,7 @@ class LimitedLocalNodeFirstLocalBalancingPolicy implements LoadBalancingPolicy
 
         Collections.shuffle(remote);
 
-        logger.debug("Using the following hosts order for the new query plan: {} | {}", local, remote);
+        logger.trace("Using the following hosts order for the new query plan: {} | {}", local, remote);
 
         return Iterators.concat(local.iterator(), remote.iterator());
     }
@@ -138,7 +138,7 @@ class LimitedLocalNodeFirstLocalBalancingPolicy implements LoadBalancingPolicy
         if (replicaAddresses.contains(host.getAddress()))
         {
             liveReplicaHosts.add(host);
-            logger.debug("Added a new host {}", host);
+            logger.trace("Added a new host {}", host);
         }
     }
 
@@ -148,7 +148,7 @@ class LimitedLocalNodeFirstLocalBalancingPolicy implements LoadBalancingPolicy
         if (replicaAddresses.contains(host.getAddress()))
         {
             liveReplicaHosts.add(host);
-            logger.debug("The host {} is now up", host);
+            logger.trace("The host {} is now up", host);
         }
     }
 
@@ -157,7 +157,7 @@ class LimitedLocalNodeFirstLocalBalancingPolicy implements LoadBalancingPolicy
     {
         if (liveReplicaHosts.remove(host))
         {
-            logger.debug("The host {} is now down", host);
+            logger.trace("The host {} is now down", host);
         }
     }
 
@@ -167,7 +167,7 @@ class LimitedLocalNodeFirstLocalBalancingPolicy implements LoadBalancingPolicy
     {
         if (liveReplicaHosts.remove(host))
         {
-            logger.debug("Removed the host {}", host);
+            logger.trace("Removed the host {}", host);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index 5d354a7..71fe037 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@ -1028,7 +1028,7 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
                 ColumnDef cDef = new ColumnDef();
                 String columnName = def.name.toString();
                 String type = def.type.toString();
-                logger.debug("name: {}, type: {} ", columnName, type);
+                logger.trace("name: {}, type: {} ", columnName, type);
                 cDef.name = ByteBufferUtil.bytes(columnName);
                 cDef.validation_class = type;
                 columnDefs.add(cDef);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java
index 223a848..bdf99a0 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java
@@ -626,7 +626,7 @@ public class CqlNativeStorage extends LoadFunc implements StoreFuncInterface, Lo
 
         if (wc != null)
         {
-            logger.debug("where clause: {}", wc);
+            logger.trace("where clause: {}", wc);
             CqlConfigHelper.setInputWhereClauses(conf, wc);
         }
         if (System.getenv(StorageHelper.PIG_INPUT_SPLIT_SIZE) != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index b07b2f5..1dd3a4e 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -276,7 +276,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         for (SSTableReader sstable : Iterables.concat(compacting, redistribute))
             total += sstable.getIndexSummaryOffHeapSize();
 
-        logger.debug("Beginning redistribution of index summaries for {} sstables with memory pool size {} MB; current spaced used is {} MB",
+        logger.trace("Beginning redistribution of index summaries for {} sstables with memory pool size {} MB; current spaced used is {} MB",
                      redistribute.size(), memoryPoolBytes / 1024L / 1024L, total / 1024.0 / 1024.0);
 
         final Map<SSTableReader, Double> readRates = new HashMap<>(redistribute.size());
@@ -310,7 +310,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         total = 0;
         for (SSTableReader sstable : Iterables.concat(compacting, oldFormatSSTables, newSSTables))
             total += sstable.getIndexSummaryOffHeapSize();
-        logger.debug("Completed resizing of index summaries; current approximate memory used: {} MB",
+        logger.trace("Completed resizing of index summaries; current approximate memory used: {} MB",
                      total / 1024.0 / 1024.0);
 
         return newSSTables;
@@ -368,7 +368,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
             if (effectiveIndexInterval < minIndexInterval)
             {
                 // The min_index_interval was changed; re-sample to match it.
-                logger.debug("Forcing resample of {} because the current index interval ({}) is below min_index_interval ({})",
+                logger.trace("Forcing resample of {} because the current index interval ({}) is below min_index_interval ({})",
                         sstable, effectiveIndexInterval, minIndexInterval);
                 long spaceUsed = (long) Math.ceil(avgEntrySize * numEntriesAtNewSamplingLevel);
                 forceResample.add(new ResampleEntry(sstable, spaceUsed, newSamplingLevel));
@@ -377,7 +377,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
             else if (effectiveIndexInterval > maxIndexInterval)
             {
                 // The max_index_interval was lowered; force an upsample to the effective minimum sampling level
-                logger.debug("Forcing upsample of {} because the current index interval ({}) is above max_index_interval ({})",
+                logger.trace("Forcing upsample of {} because the current index interval ({}) is above max_index_interval ({})",
                         sstable, effectiveIndexInterval, maxIndexInterval);
                 newSamplingLevel = Math.max(1, (BASE_SAMPLING_LEVEL * minIndexInterval) / maxIndexInterval);
                 numEntriesAtNewSamplingLevel = IndexSummaryBuilder.entriesAtSamplingLevel(newSamplingLevel, sstable.getMaxIndexSummarySize());
@@ -424,7 +424,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         for (ResampleEntry entry : toDownsample)
         {
             SSTableReader sstable = entry.sstable;
-            logger.debug("Re-sampling index summary for {} from {}/{} to {}/{} of the original number of entries",
+            logger.trace("Re-sampling index summary for {} from {}/{} to {}/{} of the original number of entries",
                          sstable, sstable.getIndexSummarySamplingLevel(), Downsampling.BASE_SAMPLING_LEVEL,
                          entry.newSamplingLevel, Downsampling.BASE_SAMPLING_LEVEL);
             ColumnFamilyStore cfs = Keyspace.open(sstable.metadata.ksName).getColumnFamilyStore(sstable.metadata.cfId);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 2077152..b0aa89e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -115,7 +115,7 @@ public abstract class SSTable
         }
         FileUtils.delete(desc.filenameFor(Component.SUMMARY));
 
-        logger.debug("Deleted {}", desc);
+        logger.trace("Deleted {}", desc);
         return true;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index ce12206..87891ae 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -296,7 +296,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                 if (cardinality != null)
                     cardinalities.add(cardinality);
                 else
-                    logger.debug("Got a null cardinality estimator in: {}", sstable.getFilename());
+                    logger.trace("Got a null cardinality estimator in: {}", sstable.getFilename());
             }
             catch (IOException e)
             {
@@ -312,7 +312,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
             return 1;
 
         long totalKeyCountAfter = mergeCardinalities(cardinalities).cardinality();
-        logger.debug("Estimated compaction gain: {}/{}={}", totalKeyCountAfter, totalKeyCountBefore, ((double)totalKeyCountAfter)/totalKeyCountBefore);
+        logger.trace("Estimated compaction gain: {}/{}={}", totalKeyCountAfter, totalKeyCountBefore, ((double)totalKeyCountAfter)/totalKeyCountBefore);
         return ((double)totalKeyCountAfter)/totalKeyCountBefore;
     }
 
@@ -399,7 +399,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
             System.exit(1);
         }
 
-        logger.info("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
+        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
         SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
                 statsMetadata, OpenReason.NORMAL);
 
@@ -446,7 +446,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
             System.exit(1);
         }
 
-        logger.info("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
+        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
         SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
                                              statsMetadata, OpenReason.NORMAL);
         try
@@ -454,14 +454,14 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
             // load index and filter
             long start = System.nanoTime();
             sstable.load(validationMetadata);
-            logger.debug("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+            logger.trace("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 
             sstable.setup(trackHotness);
             if (validate)
                 sstable.validate();
 
             if (sstable.getKeyCache() != null)
-                logger.debug("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());
+                logger.trace("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());
 
             return sstable;
         }
@@ -843,7 +843,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         {
             if (indexSummary != null)
                 indexSummary.close();
-            logger.debug("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
             // corrupted; delete it and fall back to creating a new summary
             FileUtils.closeQuietly(iStream);
             // delete it and fall back to creating a new summary
@@ -945,7 +945,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         }
         catch (IOException e)
         {
-            logger.debug("Cannot save SSTable Summary: ", e);
+            logger.trace("Cannot save SSTable Summary: ", e);
 
             // corrupted hence delete it and let it load it now.
             if (summariesFile.exists())
@@ -1633,8 +1633,8 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
      */
     public boolean markObsolete(Tracker tracker)
     {
-        if (logger.isDebugEnabled())
-            logger.debug("Marking {} compacted", getFilename());
+        if (logger.isTraceEnabled())
+            logger.trace("Marking {} compacted", getFilename());
 
         synchronized (tidy.global)
         {
@@ -1655,8 +1655,8 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
     public void markSuspect()
     {
-        if (logger.isDebugEnabled())
-            logger.debug("Marking {} as a suspect for blacklisting.", getFilename());
+        if (logger.isTraceEnabled())
+            logger.trace("Marking {} as a suspect for blacklisting.", getFilename());
 
         isSuspect.getAndSet(true);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
index 8a65d8d..30ed85b 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
@@ -78,11 +78,11 @@ public class MetadataSerializer implements IMetadataSerializer
     public Map<MetadataType, MetadataComponent> deserialize(Descriptor descriptor, EnumSet<MetadataType> types) throws IOException
     {
         Map<MetadataType, MetadataComponent> components;
-        logger.debug("Load metadata for {}", descriptor);
+        logger.trace("Load metadata for {}", descriptor);
         File statsFile = new File(descriptor.filenameFor(Component.STATS));
         if (!statsFile.exists())
         {
-            logger.debug("No sstable stats for {}", descriptor);
+            logger.trace("No sstable stats for {}", descriptor);
             components = Maps.newHashMap();
             components.put(MetadataType.STATS, MetadataCollector.defaultStatsMetadata());
         }
@@ -129,7 +129,7 @@ public class MetadataSerializer implements IMetadataSerializer
 
     public void mutateLevel(Descriptor descriptor, int newLevel) throws IOException
     {
-        logger.debug("Mutating {} to level {}", descriptor.filenameFor(Component.STATS), newLevel);
+        logger.trace("Mutating {} to level {}", descriptor.filenameFor(Component.STATS), newLevel);
         Map<MetadataType, MetadataComponent> currentComponents = deserialize(descriptor, EnumSet.allOf(MetadataType.class));
         StatsMetadata stats = (StatsMetadata) currentComponents.remove(MetadataType.STATS);
         // mutate level
@@ -139,7 +139,7 @@ public class MetadataSerializer implements IMetadataSerializer
 
     public void mutateRepairedAt(Descriptor descriptor, long newRepairedAt) throws IOException
     {
-        logger.debug("Mutating {} to repairedAt time {}", descriptor.filenameFor(Component.STATS), newRepairedAt);
+        logger.trace("Mutating {} to repairedAt time {}", descriptor.filenameFor(Component.STATS), newRepairedAt);
         Map<MetadataType, MetadataComponent> currentComponents = deserialize(descriptor, EnumSet.allOf(MetadataType.class));
         StatsMetadata stats = (StatsMetadata) currentComponents.remove(MetadataType.STATS);
         // mutate level

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 9f5193b..ce45370 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -170,8 +170,8 @@ public class FileUtils
     public static void renameWithConfirm(File from, File to)
     {
         assert from.exists();
-        if (logger.isDebugEnabled())
-            logger.debug((String.format("Renaming %s to %s", from.getPath(), to.getPath())));
+        if (logger.isTraceEnabled())
+            logger.trace((String.format("Renaming %s to %s", from.getPath(), to.getPath())));
         // this is not FSWE because usually when we see it it's because we didn't close the file before renaming it,
         // and Windows is picky about that.
         try
@@ -198,7 +198,7 @@ public class FileUtils
         }
         catch (AtomicMoveNotSupportedException e)
         {
-            logger.debug("Could not do an atomic move", e);
+            logger.trace("Could not do an atomic move", e);
             Files.move(from, to, StandardCopyOption.REPLACE_EXISTING);
         }
 
@@ -393,7 +393,7 @@ public class FileUtils
                 deleteRecursiveOnExit(new File(dir, child));
         }
 
-        logger.debug("Scheduling deferred deletion of file: " + dir);
+        logger.trace("Scheduling deferred deletion of file: " + dir);
         dir.deleteOnExit();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index 91908c9..a8fae9f 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@ -124,7 +124,7 @@ public class MmappedSegmentedFile extends SegmentedFile
                         continue;
                     FileUtils.clean(segment.right);
                 }
-                logger.debug("All segments have been unmapped successfully");
+                logger.trace("All segments have been unmapped successfully");
             }
             catch (Exception e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index 461265c..daa4aa0 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -84,7 +84,7 @@ public abstract class AbstractReplicationStrategy
             {
                 if (lastVersion > lastInvalidatedVersion)
                 {
-                    logger.debug("clearing cached endpoints");
+                    logger.trace("clearing cached endpoints");
                     cachedEndpoints.clear();
                     lastInvalidatedVersion = lastVersion;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 6959f29..307a07f 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -71,7 +71,7 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         }
 
         datacenters = Collections.unmodifiableMap(newDatacenters);
-        logger.debug("Configured datacenter replicas are {}", FBUtilities.toString(datacenters));
+        logger.trace("Configured datacenter replicas are {}", FBUtilities.toString(datacenters));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
index 4bcdcc5..590117d 100644
--- a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
@@ -99,7 +99,7 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
         String[] value = endpointMap.get(endpoint);
         if (value == null)
         {
-            logger.debug("Could not find end point information for {}, will use default", endpoint);
+            logger.trace("Could not find end point information for {}, will use default", endpoint);
             return defaultDCRack;
         }
         return value;
@@ -182,12 +182,12 @@ public class PropertyFileSnitch extends AbstractNetworkTopologySnitch
             throw new ConfigurationException(String.format("Snitch definitions at %s do not define a location for this node's broadcast address %s, nor does it provides a default",
                                                            SNITCH_PROPERTIES_FILENAME, FBUtilities.getBroadcastAddress()));
 
-        if (logger.isDebugEnabled())
+        if (logger.isTraceEnabled())
         {
             StringBuilder sb = new StringBuilder();
             for (Map.Entry<InetAddress, String[]> entry : reloadedMap.entrySet())
                 sb.append(entry.getKey()).append(":").append(Arrays.toString(entry.getValue())).append(", ");
-            logger.debug("Loaded network topology from property file: {}", StringUtils.removeEnd(sb.toString(), ", "));
+            logger.trace("Loaded network topology from property file: {}", StringUtils.removeEnd(sb.toString(), ", "));
         }
 
         endpointMap = reloadedMap;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
index 3277af7..8b34fc0 100644
--- a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
+++ b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
@@ -63,7 +63,7 @@ public class ReconnectableSnitchHelper implements IEndpointStateChangeSubscriber
                 && !MessagingService.instance().getConnectionPool(publicAddress).endPoint().equals(localAddress))
         {
             MessagingService.instance().getConnectionPool(publicAddress).reset(localAddress);
-            logger.debug(String.format("Intiated reconnect to an Internal IP %s for the %s", localAddress, publicAddress));
+            logger.trace(String.format("Intiated reconnect to an Internal IP %s for the %s", localAddress, publicAddress));
         }
     }
     

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/locator/TokenMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index fc65dd8..db0b609 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -737,8 +737,8 @@ public class TokenMetadata
 
             if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && movingEndpoints.isEmpty())
             {
-                if (logger.isDebugEnabled())
-                    logger.debug("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", keyspaceName);
+                if (logger.isTraceEnabled())
+                    logger.trace("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", keyspaceName);
 
                 pendingRanges.put(keyspaceName, newPendingRanges);
                 return;
@@ -802,8 +802,8 @@ public class TokenMetadata
 
             pendingRanges.put(keyspaceName, newPendingRanges);
 
-            if (logger.isDebugEnabled())
-                logger.debug("Pending ranges:\n{}", (pendingRanges.isEmpty() ? "<empty>" : printPendingRanges()));
+            if (logger.isTraceEnabled())
+                logger.trace("Pending ranges:\n{}", (pendingRanges.isEmpty() ? "<empty>" : printPendingRanges()));
         }
         finally
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
index 274e47b..b69ce88 100644
--- a/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
@@ -70,7 +70,7 @@ public class IncomingStreamingConnection extends Thread implements Closeable
         }
         catch (IOException e)
         {
-            logger.debug("IOException reading from socket; closing", e);
+            logger.trace("IOException reading from socket; closing", e);
             close();
         }
     }
@@ -87,7 +87,7 @@ public class IncomingStreamingConnection extends Thread implements Closeable
         }
         catch (IOException e)
         {
-            logger.debug("Error closing socket", e);
+            logger.trace("Error closing socket", e);
         }
         finally
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
index cfdb854..f6652b0 100644
--- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
@@ -101,7 +101,7 @@ public class IncomingTcpConnection extends Thread implements Closeable
         }
         catch (IOException e)
         {
-            logger.debug("IOException reading from socket; closing", e);
+            logger.trace("IOException reading from socket; closing", e);
         }
         finally
         {
@@ -121,7 +121,7 @@ public class IncomingTcpConnection extends Thread implements Closeable
         }
         catch (IOException e)
         {
-            logger.debug("Error closing socket", e);
+            logger.trace("Error closing socket", e);
         }
         finally
         {
@@ -144,11 +144,11 @@ public class IncomingTcpConnection extends Thread implements Closeable
         from = CompactEndpointSerializationHelper.deserialize(in);
         // record the (true) version of the endpoint
         MessagingService.instance().setVersion(from, maxVersion);
-        logger.debug("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from));
+        logger.trace("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from));
 
         if (compressed)
         {
-            logger.debug("Upgrading incoming connection to be compressed");
+            logger.trace("Upgrading incoming connection to be compressed");
             if (version < MessagingService.VERSION_21)
             {
                 in = new DataInputStream(new SnappyInputStream(socket.getInputStream()));
@@ -206,7 +206,7 @@ public class IncomingTcpConnection extends Thread implements Closeable
         }
         else
         {
-            logger.debug("Received connection from newer protocol version {}. Ignoring message", version);
+            logger.trace("Received connection from newer protocol version {}. Ignoring message", version);
         }
         return message.from;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
index 0ff0319..a46366c 100644
--- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
+++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
@@ -57,7 +57,7 @@ public class MessageDeliveryTask implements Runnable
         IVerbHandler verbHandler = MessagingService.instance().getVerbHandler(verb);
         if (verbHandler == null)
         {
-            logger.debug("Unknown verb {}", verb);
+            logger.trace("Unknown verb {}", verb);
             return;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 1f3240d..2a63553 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -443,7 +443,7 @@ public final class MessagingService implements MessagingServiceMBean
      */
     public void convict(InetAddress ep)
     {
-        logger.debug("Resetting pool for {}", ep);
+        logger.trace("Resetting pool for {}", ep);
         getConnectionPool(ep).reset();
     }
 
@@ -538,7 +538,7 @@ public final class MessagingService implements MessagingServiceMBean
         }
         catch (InterruptedException ie)
         {
-            logger.debug("await interrupted");
+            logger.trace("await interrupted");
         }
     }
 
@@ -831,7 +831,7 @@ public final class MessagingService implements MessagingServiceMBean
      */
     public int setVersion(InetAddress endpoint, int version)
     {
-        logger.debug("Setting version {} for {}", version, endpoint);
+        logger.trace("Setting version {} for {}", version, endpoint);
 
         if (version < VERSION_22)
             allNodesAtLeast22 = false;
@@ -847,7 +847,7 @@ public final class MessagingService implements MessagingServiceMBean
 
     public void resetVersion(InetAddress endpoint)
     {
-        logger.debug("Resetting version for {}", endpoint);
+        logger.trace("Resetting version for {}", endpoint);
         Integer removed = versions.remove(endpoint);
         if (removed != null && removed <= VERSION_22)
             refreshAllNodesAtLeast22();
@@ -972,7 +972,7 @@ public final class MessagingService implements MessagingServiceMBean
                     socket = server.accept();
                     if (!authenticate(socket))
                     {
-                        logger.debug("remote failed to authenticate");
+                        logger.trace("remote failed to authenticate");
                         socket.close();
                         continue;
                     }
@@ -985,7 +985,7 @@ public final class MessagingService implements MessagingServiceMBean
                     int header = in.readInt();
                     boolean isStream = MessagingService.getBits(header, 3, 1) == 1;
                     int version = MessagingService.getBits(header, 15, 8);
-                    logger.debug("Connection version {} from {}", version, socket.getInetAddress());
+                    logger.trace("Connection version {} from {}", version, socket.getInetAddress());
                     socket.setSoTimeout(0);
 
                     Thread thread = isStream
@@ -997,17 +997,17 @@ public final class MessagingService implements MessagingServiceMBean
                 catch (AsynchronousCloseException e)
                 {
                     // this happens when another thread calls close().
-                    logger.debug("Asynchronous close seen by server thread");
+                    logger.trace("Asynchronous close seen by server thread");
                     break;
                 }
                 catch (ClosedChannelException e)
                 {
-                    logger.debug("MessagingService server thread already closed");
+                    logger.trace("MessagingService server thread already closed");
                     break;
                 }
                 catch (IOException e)
                 {
-                    logger.debug("Error reading the socket " + socket, e);
+                    logger.trace("Error reading the socket " + socket, e);
                     FileUtils.closeQuietly(socket);
                 }
             }
@@ -1016,7 +1016,7 @@ public final class MessagingService implements MessagingServiceMBean
 
         void close() throws IOException
         {
-            logger.debug("Closing accept() thread");
+            logger.trace("Closing accept() thread");
 
             try
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
index cb168bc..31a5269 100644
--- a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
@@ -295,8 +295,8 @@ public class OutboundTcpConnection extends Thread
             disconnect();
             if (e instanceof IOException || e.getCause() instanceof IOException)
             {
-                if (logger.isDebugEnabled())
-                    logger.debug("error writing to {}", poolReference.endPoint(), e);
+                if (logger.isTraceEnabled())
+                    logger.trace("error writing to {}", poolReference.endPoint(), e);
 
                 // if the message was important, such as a repair acknowledgement, put it back on the queue
                 // to retry after re-connecting.  See CASSANDRA-5393
@@ -371,8 +371,8 @@ public class OutboundTcpConnection extends Thread
     @SuppressWarnings("resource")
     private boolean connect()
     {
-        if (logger.isDebugEnabled())
-            logger.debug("attempting to connect to {}", poolReference.endPoint());
+        if (logger.isTraceEnabled())
+            logger.trace("attempting to connect to {}", poolReference.endPoint());
 
         long start = System.nanoTime();
         long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getRpcTimeout());
@@ -418,7 +418,7 @@ public class OutboundTcpConnection extends Thread
                     // no version is returned, so disconnect an try again: we will either get
                     // a different target version (targetVersion < MessagingService.VERSION_12)
                     // or if the same version the handshake will finally succeed
-                    logger.debug("Target max version is {}; no version information yet, will retry", maxTargetVersion);
+                    logger.trace("Target max version is {}; no version information yet, will retry", maxTargetVersion);
                     if (DatabaseDescriptor.getSeeds().contains(poolReference.endPoint()))
                         logger.warn("Seed gossip version is {}; will not connect with that version", maxTargetVersion);
                     disconnect();
@@ -431,7 +431,7 @@ public class OutboundTcpConnection extends Thread
 
                 if (targetVersion > maxTargetVersion)
                 {
-                    logger.debug("Target max version is {}; will reconnect with that version", maxTargetVersion);
+                    logger.trace("Target max version is {}; will reconnect with that version", maxTargetVersion);
                     disconnect();
                     return false;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
index 1d9aa98..28ed365 100644
--- a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
+++ b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
@@ -35,7 +35,7 @@ public class ResponseVerbHandler implements IVerbHandler
         if (callbackInfo == null)
         {
             String msg = "Callback already removed for {} (from {})";
-            logger.debug(msg, id, message.from);
+            logger.trace(msg, id, message.from);
             Tracing.trace(msg, id, message.from);
             return;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaTables.java b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
index 30fe013..5eae60b 100644
--- a/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
@@ -1357,7 +1357,7 @@ public class LegacySchemaTables
                 udf.body().equals(body) &&
                 udf.isCalledOnNullInput() == calledOnNullInput)
             {
-                logger.debug("Skipping duplicate compilation of already existing UDF {}", name);
+                logger.trace("Skipping duplicate compilation of already existing UDF {}", name);
                 return udf;
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/service/FileCacheService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/FileCacheService.java b/src/java/org/apache/cassandra/service/FileCacheService.java
index 1e12310..19d6a70 100644
--- a/src/java/org/apache/cassandra/service/FileCacheService.java
+++ b/src/java/org/apache/cassandra/service/FileCacheService.java
@@ -147,8 +147,8 @@ public class FileCacheService
     public void put(CacheKey cacheKey, RandomAccessReader instance)
     {
         int memoryUsed = memoryUsage.get();
-        if (logger.isDebugEnabled())
-            logger.debug("Estimated memory usage is {} compared to actual usage {}", memoryUsed, sizeInBytes());
+        if (logger.isTraceEnabled())
+            logger.trace("Estimated memory usage is {} compared to actual usage {}", memoryUsed, sizeInBytes());
 
         CacheBucket bucket = cache.getIfPresent(cacheKey);
         if (memoryUsed >= MEMORY_USAGE_THRESHOLD || bucket == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/service/GCInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java
index 3a4ec22..89a4a7e 100644
--- a/src/java/org/apache/cassandra/service/GCInspector.java
+++ b/src/java/org/apache/cassandra/service/GCInspector.java
@@ -282,8 +282,8 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
                 logger.warn(st);
             else if (duration > MIN_LOG_DURATION)
                 logger.info(st);
-            else if (logger.isDebugEnabled())
-                logger.debug(st);
+            else if (logger.isTraceEnabled())
+                logger.trace(st);
 
             if (duration > STAT_THRESHOLD)
                 StatusLogger.log();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/service/LoadBroadcaster.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/LoadBroadcaster.java b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
index 238daae..69fa93d 100644
--- a/src/java/org/apache/cassandra/service/LoadBroadcaster.java
+++ b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
@@ -87,8 +87,8 @@ public class LoadBroadcaster implements IEndpointStateChangeSubscriber
         {
             public void run()
             {
-                if (logger.isDebugEnabled())
-                    logger.debug("Disseminating load info ...");
+                if (logger.isTraceEnabled())
+                    logger.trace("Disseminating load info ...");
                 Gossiper.instance.addLocalApplicationState(ApplicationState.LOAD,
                                                            StorageService.instance.valueFactory.load(StorageMetrics.load.getCount()));
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/service/ReadCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ReadCallback.java b/src/java/org/apache/cassandra/service/ReadCallback.java
index 57d8064..145679d 100644
--- a/src/java/org/apache/cassandra/service/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/ReadCallback.java
@@ -109,8 +109,8 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallbackWithFail
             // Same as for writes, see AbstractWriteResponseHandler
             ReadTimeoutException ex = new ReadTimeoutException(consistencyLevel, received, blockfor, resolver.isDataPresent());
             Tracing.trace("Read timeout: {}", ex.toString());
-            if (logger.isDebugEnabled())
-                logger.debug("Read timeout: {}", ex.toString());
+            if (logger.isTraceEnabled())
+                logger.trace("Read timeout: {}", ex.toString());
             throw ex;
         }
 
@@ -118,8 +118,8 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallbackWithFail
         {
             ReadFailureException ex = new ReadFailureException(consistencyLevel, received, failures, blockfor, resolver.isDataPresent());
 
-            if (logger.isDebugEnabled())
-                logger.debug("Read failure: {}", ex.toString());
+            if (logger.isTraceEnabled())
+                logger.trace("Read failure: {}", ex.toString());
             throw ex;
         }
 
@@ -210,8 +210,8 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallbackWithFail
 
                 if (traceState != null)
                     traceState.trace("Digest mismatch: {}", e.toString());
-                if (logger.isDebugEnabled())
-                    logger.debug("Digest mismatch:", e);
+                if (logger.isTraceEnabled())
+                    logger.trace("Digest mismatch:", e);
                 
                 ReadRepairMetrics.repairedBackground.mark();
                 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/service/RowDataResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/RowDataResolver.java b/src/java/org/apache/cassandra/service/RowDataResolver.java
index e935ce7..e15302b 100644
--- a/src/java/org/apache/cassandra/service/RowDataResolver.java
+++ b/src/java/org/apache/cassandra/service/RowDataResolver.java
@@ -59,8 +59,8 @@ public class RowDataResolver extends AbstractRowResolver
     public Row resolve() throws DigestMismatchException
     {
         int replyCount = replies.size();
-        if (logger.isDebugEnabled())
-            logger.debug("resolving {} responses", replyCount);
+        if (logger.isTraceEnabled())
+            logger.trace("resolving {} responses", replyCount);
         long start = System.nanoTime();
 
         ColumnFamily resolved;
@@ -84,8 +84,8 @@ public class RowDataResolver extends AbstractRowResolver
             }
 
             resolved = resolveSuperset(versions, timestamp);
-            if (logger.isDebugEnabled())
-                logger.debug("versions merged");
+            if (logger.isTraceEnabled())
+                logger.trace("versions merged");
 
             // send updates to any replica that was missing part of the full row
             // (resolved can be null even if versions doesn't have all nulls because of the call to removeDeleted in resolveSuperSet)
@@ -97,8 +97,8 @@ public class RowDataResolver extends AbstractRowResolver
             resolved = replies.get(0).payload.row().cf;
         }
 
-        if (logger.isDebugEnabled())
-            logger.debug("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+        if (logger.isTraceEnabled())
+            logger.trace("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 
         return new Row(key, resolved);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/service/RowDigestResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/RowDigestResolver.java b/src/java/org/apache/cassandra/service/RowDigestResolver.java
index 95b5b80..32b26e1 100644
--- a/src/java/org/apache/cassandra/service/RowDigestResolver.java
+++ b/src/java/org/apache/cassandra/service/RowDigestResolver.java
@@ -63,8 +63,8 @@ public class RowDigestResolver extends AbstractRowResolver
      */
     public Row resolve() throws DigestMismatchException
     {
-        if (logger.isDebugEnabled())
-            logger.debug("resolving {} responses", replies.size());
+        if (logger.isTraceEnabled())
+            logger.trace("resolving {} responses", replies.size());
 
         long start = System.nanoTime();
 
@@ -98,8 +98,8 @@ public class RowDigestResolver extends AbstractRowResolver
                 throw new DigestMismatchException(key, digest, newDigest);
         }
 
-        if (logger.isDebugEnabled())
-            logger.debug("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+        if (logger.isTraceEnabled())
+            logger.trace("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
         return new Row(key, data);
     }
 


[6/9] cassandra git commit: Merge branch cassandra-2.2 into cassandra-3.0

Posted by bl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
index 1f0191a,0000000..1dadf20
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
@@@ -1,265 -1,0 +1,265 @@@
 +/*
 + * 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.hints;
 +
 +import java.io.File;
 +import java.util.Map;
 +import java.util.UUID;
 +import java.util.concurrent.*;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.function.Supplier;
 +
 +import com.google.common.util.concurrent.RateLimiter;
 +
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.io.FSReadError;
 +import org.apache.cassandra.service.StorageService;
 +
 +/**
 + * A multi-threaded (by default) executor for dispatching hints.
 + *
 + * Most of dispatch is triggered by {@link HintsDispatchTrigger} running every ~10 seconds.
 + */
 +final class HintsDispatchExecutor
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(HintsDispatchExecutor.class);
 +
 +    private final File hintsDirectory;
 +    private final ExecutorService executor;
 +    private final AtomicBoolean isPaused;
 +    private final Map<UUID, Future> scheduledDispatches;
 +
 +    HintsDispatchExecutor(File hintsDirectory, int maxThreads, AtomicBoolean isPaused)
 +    {
 +        this.hintsDirectory = hintsDirectory;
 +        this.isPaused = isPaused;
 +
 +        scheduledDispatches = new ConcurrentHashMap<>();
 +        executor = new JMXEnabledThreadPoolExecutor(1,
 +                                                    maxThreads,
 +                                                    1,
 +                                                    TimeUnit.MINUTES,
 +                                                    new LinkedBlockingQueue<>(),
 +                                                    new NamedThreadFactory("HintsDispatcher", Thread.MIN_PRIORITY),
 +                                                    "internal");
 +    }
 +
 +    /*
 +     * It's safe to terminate dispatch in process and to deschedule dispatch.
 +     */
 +    void shutdownBlocking()
 +    {
 +        scheduledDispatches.clear();
 +        executor.shutdownNow();
 +    }
 +
 +    boolean isScheduled(HintsStore store)
 +    {
 +        return scheduledDispatches.containsKey(store.hostId);
 +    }
 +
 +    Future dispatch(HintsStore store)
 +    {
 +        return dispatch(store, store.hostId);
 +    }
 +
 +    Future dispatch(HintsStore store, UUID hostId)
 +    {
 +        /*
 +         * It is safe to perform dispatch for the same host id concurrently in two or more threads,
 +         * however there is nothing to win from it - so we don't.
 +         *
 +         * Additionally, having just one dispatch task per host id ensures that we'll never violate our per-destination
 +         * rate limit, without having to share a ratelimiter between threads.
 +         *
 +         * It also simplifies reasoning about dispatch sessions.
 +         */
 +        return scheduledDispatches.computeIfAbsent(hostId, uuid -> executor.submit(new DispatchHintsTask(store, hostId)));
 +    }
 +
 +    Future transfer(HintsCatalog catalog, Supplier<UUID> hostIdSupplier)
 +    {
 +        return executor.submit(new TransferHintsTask(catalog, hostIdSupplier));
 +    }
 +
 +    void completeDispatchBlockingly(HintsStore store)
 +    {
 +        Future future = scheduledDispatches.get(store.hostId);
 +        try
 +        {
 +            if (future != null)
 +                future.get();
 +        }
 +        catch (ExecutionException | InterruptedException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    private final class TransferHintsTask implements Runnable
 +    {
 +        private final HintsCatalog catalog;
 +
 +        /*
 +         * Supplies target hosts to stream to. Generally returns the one the DynamicSnitch thinks is closest.
 +         * We use a supplier here to be able to get a new host if the current one dies during streaming.
 +         */
 +        private final Supplier<UUID> hostIdSupplier;
 +
 +        private TransferHintsTask(HintsCatalog catalog, Supplier<UUID> hostIdSupplier)
 +        {
 +            this.catalog = catalog;
 +            this.hostIdSupplier = hostIdSupplier;
 +        }
 +
 +        @Override
 +        public void run()
 +        {
 +            UUID hostId = hostIdSupplier.get();
 +            logger.info("Transferring all hints to {}", hostId);
 +            if (transfer(hostId))
 +                return;
 +
 +            logger.warn("Failed to transfer all hints to {}; will retry in {} seconds", hostId, 10);
 +
 +            try
 +            {
 +                TimeUnit.SECONDS.sleep(10);
 +            }
 +            catch (InterruptedException e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +
 +            hostId = hostIdSupplier.get();
 +            logger.info("Transferring all hints to {}", hostId);
 +            if (!transfer(hostId))
 +            {
 +                logger.error("Failed to transfer all hints to {}", hostId);
 +                throw new RuntimeException("Failed to transfer all hints to " + hostId);
 +            }
 +        }
 +
 +        private boolean transfer(UUID hostId)
 +        {
 +            catalog.stores()
 +                   .map(store -> new DispatchHintsTask(store, hostId))
 +                   .forEach(Runnable::run);
 +
 +            return !catalog.hasFiles();
 +        }
 +    }
 +
 +    private final class DispatchHintsTask implements Runnable
 +    {
 +        private final HintsStore store;
 +        private final UUID hostId;
 +        private final RateLimiter rateLimiter;
 +
 +        DispatchHintsTask(HintsStore store, UUID hostId)
 +        {
 +            this.store = store;
 +            this.hostId = hostId;
 +
 +            // rate limit is in bytes per second. Uses Double.MAX_VALUE if disabled (set to 0 in cassandra.yaml).
 +            // max rate is scaled by the number of nodes in the cluster (CASSANDRA-5272).
 +            // the goal is to bound maximum hints traffic going towards a particular node from the rest of the cluster,
 +            // not total outgoing hints traffic from this node - this is why the rate limiter is not shared between
 +            // all the dispatch tasks (as there will be at most one dispatch task for a particular host id at a time).
 +            int nodesCount = Math.max(1, StorageService.instance.getTokenMetadata().getAllEndpoints().size() - 1);
 +            int throttleInKB = DatabaseDescriptor.getHintedHandoffThrottleInKB() / nodesCount;
 +            this.rateLimiter = RateLimiter.create(throttleInKB == 0 ? Double.MAX_VALUE : throttleInKB * 1024);
 +        }
 +
 +        public void run()
 +        {
 +            try
 +            {
 +                dispatch();
 +            }
 +            finally
 +            {
 +                scheduledDispatches.remove(hostId);
 +            }
 +        }
 +
 +        private void dispatch()
 +        {
 +            while (true)
 +            {
 +                if (isPaused.get())
 +                    break;
 +
 +                HintsDescriptor descriptor = store.poll();
 +                if (descriptor == null)
 +                    break;
 +
 +                try
 +                {
 +                    if (!dispatch(descriptor))
 +                        break;
 +                }
 +                catch (FSReadError e)
 +                {
 +                    logger.error("Failed to dispatch hints file {}: file is corrupted ({})", descriptor.fileName(), e);
 +                    store.cleanUp(descriptor);
 +                    store.blacklist(descriptor);
 +                    throw e;
 +                }
 +            }
 +        }
 +
 +        /*
 +         * Will return true if dispatch was successful, false if we hit a failure (destination node went down, for example).
 +         */
 +        private boolean dispatch(HintsDescriptor descriptor)
 +        {
-             logger.debug("Dispatching hints file {}", descriptor.fileName());
++            logger.trace("Dispatching hints file {}", descriptor.fileName());
 +
 +            File file = new File(hintsDirectory, descriptor.fileName());
 +            Long offset = store.getDispatchOffset(descriptor).orElse(null);
 +
 +            try (HintsDispatcher dispatcher = HintsDispatcher.create(file, rateLimiter, hostId, descriptor.hostId, isPaused))
 +            {
 +                if (offset != null)
 +                    dispatcher.seek(offset);
 +
 +                if (dispatcher.dispatch())
 +                {
 +                    if (!file.delete())
 +                        logger.error("Failed to delete hints file {}", descriptor.fileName());
 +                    store.cleanUp(descriptor);
 +                    logger.info("Finished hinted handoff of file {} to endpoint {}", descriptor.fileName(), hostId);
 +                    return true;
 +                }
 +                else
 +                {
 +                    store.markDispatchOffset(descriptor, dispatcher.dispatchOffset());
 +                    store.offerFirst(descriptor);
 +                    logger.info("Finished hinted handoff of file {} to endpoint {}, partially", descriptor.fileName(), hostId);
 +                    return false;
 +                }
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 47364f6,0000000..3daf147
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@@ -1,1002 -1,0 +1,1002 @@@
 +/*
 + * 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.index;
 +
 +import java.lang.reflect.Constructor;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.function.Function;
 +import java.util.stream.Collectors;
 +import java.util.stream.Stream;
 +
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Strings;
 +import com.google.common.collect.ImmutableSet;
 +import com.google.common.collect.Maps;
 +import com.google.common.primitives.Longs;
 +import com.google.common.util.concurrent.Futures;
 +import com.google.common.util.concurrent.MoreExecutors;
 +import org.apache.commons.lang3.StringUtils;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
 +import org.apache.cassandra.concurrent.StageManager;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.internal.CassandraIndex;
 +import org.apache.cassandra.index.transactions.*;
 +import org.apache.cassandra.io.sstable.ReducingKeyIterator;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.Indexes;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.concurrent.Refs;
 +
 +/**
 + * Handles the core maintenance functionality associated with indexes: adding/removing them to or from
 + * a table, (re)building during bootstrap or other streaming operations, flushing, reloading metadata
 + * and so on.
 + *
 + * The Index interface defines a number of methods which return Callable<?>. These are primarily the
 + * management tasks for an index implementation. Most of them are currently executed in a blocking
 + * fashion via submission to SIM's blockingExecutor. This provides the desired behaviour in pretty
 + * much all cases, as tasks like flushing an index needs to be executed synchronously to avoid potentially
 + * deadlocking on the FlushWriter or PostFlusher. Several of these Callable<?> returning methods on Index could
 + * then be defined with as void and called directly from SIM (rather than being run via the executor service).
 + * Separating the task defintion from execution gives us greater flexibility though, so that in future, for example,
 + * if the flush process allows it we leave open the possibility of executing more of these tasks asynchronously.
 + *
 + * The primary exception to the above is the Callable returned from Index#addIndexedColumn. This may
 + * involve a significant effort, building a new index over any existing data. We perform this task asynchronously;
 + * as it is called as part of a schema update, which we do not want to block for a long period. Building non-custom
 + * indexes is performed on the CompactionManager.
 + *
 + * This class also provides instances of processors which listen to updates to the base table and forward to
 + * registered Indexes the info required to keep those indexes up to date.
 + * There are two variants of these processors, each with a factory method provided by SIM:
 + *      IndexTransaction: deals with updates generated on the regular write path.
 + *      CleanupTransaction: used when partitions are modified during compaction or cleanup operations.
 + * Further details on their usage and lifecycles can be found in the interface definitions below.
 + *
 + * Finally, the bestIndexFor method is used at query time to identify the most selective index of those able
 + * to satisfy any search predicates defined by a ReadCommand's RowFilter. It returns a thin IndexAccessor object
 + * which enables the ReadCommand to access the appropriate functions of the Index at various stages in its lifecycle.
 + * e.g. the getEstimatedResultRows is required when StorageProxy calculates the initial concurrency factor for
 + * distributing requests to replicas, whereas a Searcher instance is needed when the ReadCommand is executed locally on
 + * a target replica.
 + */
 +public class SecondaryIndexManager implements IndexRegistry
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(SecondaryIndexManager.class);
 +
 +    private Map<String, Index> indexes = Maps.newConcurrentMap();
 +
 +    // executes tasks returned by Indexer#addIndexColumn which may require index(es) to be (re)built
 +    private static final ExecutorService asyncExecutor =
 +        new JMXEnabledThreadPoolExecutor(1,
 +                                         StageManager.KEEPALIVE,
 +                                         TimeUnit.SECONDS,
 +                                         new LinkedBlockingQueue<>(),
 +                                         new NamedThreadFactory("SecondaryIndexManagement"),
 +                                         "internal");
 +
 +    // executes all blocking tasks produced by Indexers e.g. getFlushTask, getMetadataReloadTask etc
 +    private static final ExecutorService blockingExecutor = MoreExecutors.newDirectExecutorService();
 +
 +    /**
 +     * The underlying column family containing the source data for these indexes
 +     */
 +    public final ColumnFamilyStore baseCfs;
 +
 +    public SecondaryIndexManager(ColumnFamilyStore baseCfs)
 +    {
 +        this.baseCfs = baseCfs;
 +    }
 +
 +
 +    /**
 +     * Drops and adds new indexes associated with the underlying CF
 +     */
 +    public void reload()
 +    {
 +        // figure out what needs to be added and dropped.
 +        Indexes tableIndexes = baseCfs.metadata.getIndexes();
 +        indexes.keySet()
 +               .stream()
 +               .filter(indexName -> !tableIndexes.has(indexName))
 +               .forEach(this::removeIndex);
 +
 +        // we call add for every index definition in the collection as
 +        // some may not have been created here yet, only added to schema
 +        for (IndexMetadata tableIndex : tableIndexes)
 +            addIndex(tableIndex);
 +    }
 +
 +    private Future<?> reloadIndex(IndexMetadata indexDef)
 +    {
 +        // if the index metadata has changed, reload the index
 +        IndexMetadata registered = indexes.get(indexDef.name).getIndexMetadata();
 +        if (!registered.equals(indexDef))
 +        {
 +            Index index = indexes.remove(registered.name);
 +            index.register(this);
 +            return blockingExecutor.submit(index.getMetadataReloadTask(indexDef));
 +        }
 +
 +        // otherwise, nothing to do
 +        return Futures.immediateFuture(null);
 +    }
 +
 +    private Future<?> createIndex(IndexMetadata indexDef)
 +    {
 +        Index index = createInstance(indexDef);
 +        index.register(this);
 +        final Callable<?> initialBuildTask = index.getInitializationTask();
 +        return initialBuildTask == null
 +               ? Futures.immediateFuture(null)
 +               : asyncExecutor.submit(initialBuildTask);
 +    }
 +
 +    /**
 +     * Adds and builds a index
 +     * @param indexDef the IndexMetadata describing the index
 +     */
 +    public synchronized Future<?> addIndex(IndexMetadata indexDef)
 +    {
 +        if (indexes.containsKey(indexDef.name))
 +            return reloadIndex(indexDef);
 +        else
 +            return createIndex(indexDef);
 +    }
 +
 +    public synchronized void removeIndex(String indexName)
 +    {
 +        Index index = indexes.remove(indexName);
 +        if (null != index)
 +        {
 +            executeBlocking(index.getInvalidateTask());
 +            unregisterIndex(index);
 +        }
 +    }
 +
 +
 +    public Set<IndexMetadata> getDependentIndexes(ColumnDefinition column)
 +    {
 +        if (indexes.isEmpty())
 +            return Collections.emptySet();
 +
 +        Set<IndexMetadata> dependentIndexes = new HashSet<>();
 +        for (Index index : indexes.values())
 +            if (index.dependsOn(column))
 +                dependentIndexes.add(index.getIndexMetadata());
 +
 +        return dependentIndexes;
 +    }
 +
 +
 +    /**
 +     * Called when dropping a Table
 +     */
 +    public void markAllIndexesRemoved()
 +    {
 +       getBuiltIndexNames().forEach(this::markIndexRemoved);
 +    }
 +
 +    /**
 +    * Does a full, blocking rebuild of the indexes specified by columns from the sstables.
 +    * Caller must acquire and release references to the sstables used here.
 +    * Note also that only this method of (re)building indexes:
 +    *   a) takes a set of index *names* rather than Indexers
 +    *   b) marks exsiting indexes removed prior to rebuilding
 +    *
 +    * @param sstables the data to build from
 +    * @param indexNames the list of indexes to be rebuilt
 +    */
 +    public void rebuildIndexesBlocking(Collection<SSTableReader> sstables, Set<String> indexNames)
 +    {
 +        Set<Index> toRebuild = indexes.values().stream()
 +                                               .filter(index -> indexNames.contains(index.getIndexName()))
 +                                               .filter(Index::shouldBuildBlocking)
 +                                               .collect(Collectors.toSet());
 +        if (toRebuild.isEmpty())
 +        {
 +            logger.info("No defined indexes with the supplied names: {}", Joiner.on(',').join(indexNames));
 +            return;
 +        }
 +
 +        toRebuild.forEach(indexer -> markIndexRemoved(indexer.getIndexName()));
 +
 +        buildIndexesBlocking(sstables, toRebuild);
 +
 +        toRebuild.forEach(indexer -> markIndexBuilt(indexer.getIndexName()));
 +    }
 +
 +    public void buildAllIndexesBlocking(Collection<SSTableReader> sstables)
 +    {
 +        buildIndexesBlocking(sstables, indexes.values()
 +                                              .stream()
 +                                              .filter(Index::shouldBuildBlocking)
 +                                              .collect(Collectors.toSet()));
 +    }
 +
 +    // For convenience, may be called directly from Index impls
 +    public void buildIndexBlocking(Index index)
 +    {
 +        if (index.shouldBuildBlocking())
 +        {
 +            try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.select(SSTableSet.CANONICAL));
 +                 Refs<SSTableReader> sstables = viewFragment.refs)
 +            {
 +                buildIndexesBlocking(sstables, Collections.singleton(index));
 +                markIndexBuilt(index.getIndexName());
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is a secondary index.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamilyStore(ColumnFamilyStore cfs)
 +    {
 +        return isIndexColumnFamily(cfs.name);
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is the one secondary index.
 +     *
 +     * @param cfs the name of the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamily(String cfName)
 +    {
 +        return cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the parent of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the parent of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static ColumnFamilyStore getParentCfs(ColumnFamilyStore cfs)
 +    {
 +        String parentCfs = getParentCfsName(cfs.name);
 +        return cfs.keyspace.getColumnFamilyStore(parentCfs);
 +    }
 +
 +    /**
 +     * Returns the parent name of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the parent name of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static String getParentCfsName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringBefore(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the index name
 +     */
 +    public static String getIndexName(ColumnFamilyStore cfs)
 +    {
 +        return getIndexName(cfs.name);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the index name
 +     */
 +    public static String getIndexName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringAfter(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    private void buildIndexesBlocking(Collection<SSTableReader> sstables, Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        logger.info("Submitting index build of {} for data in {}",
 +                    indexes.stream().map(Index::getIndexName).collect(Collectors.joining(",")),
 +                    sstables.stream().map(SSTableReader::toString).collect(Collectors.joining(",")));
 +
 +        SecondaryIndexBuilder builder = new SecondaryIndexBuilder(baseCfs,
 +                                                                  indexes,
 +                                                                  new ReducingKeyIterator(sstables));
 +        Future<?> future = CompactionManager.instance.submitIndexBuild(builder);
 +        FBUtilities.waitOnFuture(future);
 +
 +        flushIndexesBlocking(indexes);
 +        logger.info("Index build of {} complete",
 +                    indexes.stream().map(Index::getIndexName).collect(Collectors.joining(",")));
 +    }
 +
 +    private void markIndexBuilt(String indexName)
 +    {
 +        SystemKeyspace.setIndexBuilt(baseCfs.name, indexName);
 +    }
 +
 +    private void markIndexRemoved(String indexName)
 +    {
 +        SystemKeyspace.setIndexRemoved(baseCfs.name, indexName);
 +    }
 +
 +
 +    public Index getIndexByName(String indexName)
 +    {
 +        return indexes.get(indexName);
 +    }
 +
 +    private Index createInstance(IndexMetadata indexDef)
 +    {
 +        Index newIndex;
 +        if (indexDef.isCustom())
 +        {
 +            assert indexDef.options != null;
 +            String className = indexDef.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME);
 +            assert ! Strings.isNullOrEmpty(className);
 +            try
 +            {
 +                Class<? extends Index> indexClass = FBUtilities.classForName(className, "Index");
 +                Constructor ctor = indexClass.getConstructor(ColumnFamilyStore.class, IndexMetadata.class);
 +                newIndex = (Index)ctor.newInstance(baseCfs, indexDef);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +        else
 +        {
 +            newIndex = CassandraIndex.newIndex(baseCfs, indexDef);
 +        }
 +        return newIndex;
 +    }
 +
 +    /**
 +     * Truncate all indexes
 +     */
 +    public void truncateAllIndexesBlocking(final long truncatedAt)
 +    {
 +        executeAllBlocking(indexes.values().stream(), (index) -> index.getTruncateTask(truncatedAt));
 +    }
 +
 +    /**
 +     * Remove all indexes
 +     */
 +    public void invalidateAllIndexesBlocking()
 +    {
 +        executeAllBlocking(indexes.values().stream(), Index::getInvalidateTask);
 +    }
 +
 +    /**
 +     * Perform a blocking flush all indexes
 +     */
 +    public void flushAllIndexesBlocking()
 +    {
 +       flushIndexesBlocking(ImmutableSet.copyOf(indexes.values()));
 +    }
 +
 +    /**
 +     * Perform a blocking flush of selected indexes
 +     */
 +    public void flushIndexesBlocking(Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        List<Future<?>> wait = new ArrayList<>();
 +        List<Index> nonCfsIndexes = new ArrayList<>();
 +
 +        // for each CFS backed index, submit a flush task which we'll wait on for completion
 +        // for the non-CFS backed indexes, we'll flush those while we wait.
 +        synchronized (baseCfs.getTracker())
 +        {
 +            indexes.forEach(index ->
 +                index.getBackingTable()
 +                     .map(cfs -> wait.add(cfs.forceFlush()))
 +                     .orElseGet(() -> nonCfsIndexes.add(index)));
 +        }
 +
 +        executeAllBlocking(nonCfsIndexes.stream(), Index::getBlockingFlushTask);
 +        FBUtilities.waitOnFutures(wait);
 +    }
 +
 +    /**
 +     * Performs a blocking flush of all custom indexes
 +     */
 +    public void flushAllNonCFSBackedIndexesBlocking()
 +    {
 +        Set<Index> customIndexers = indexes.values().stream()
 +                                                    .filter(index -> !(index.getBackingTable().isPresent()))
 +                                                    .collect(Collectors.toSet());
 +        flushIndexesBlocking(customIndexers);
 +    }
 +
 +    /**
 +     * @return all indexes which are marked as built and ready to use
 +     */
 +    public List<String> getBuiltIndexNames()
 +    {
 +        Set<String> allIndexNames = new HashSet<>();
 +        indexes.values().stream()
 +                .map(Index::getIndexName)
 +                .forEach(allIndexNames::add);
 +        return SystemKeyspace.getBuiltIndexes(baseCfs.keyspace.getName(), allIndexNames);
 +    }
 +
 +    /**
 +     * @return all backing Tables used by registered indexes
 +     */
 +    public Set<ColumnFamilyStore> getAllIndexColumnFamilyStores()
 +    {
 +        Set<ColumnFamilyStore> backingTables = new HashSet<>();
 +        indexes.values().forEach(index -> index.getBackingTable().ifPresent(backingTables::add));
 +        return backingTables;
 +    }
 +
 +    /**
 +     * @return if there are ANY indexes registered for this table
 +     */
 +    public boolean hasIndexes()
 +    {
 +        return !indexes.isEmpty();
 +    }
 +
 +    /**
 +     * When building an index against existing data in sstables, add the given partition to the index
 +     */
 +    public void indexPartition(UnfilteredRowIterator partition, OpOrder.Group opGroup, Set<Index> indexes, int nowInSec)
 +    {
 +        if (!indexes.isEmpty())
 +        {
 +            DecoratedKey key = partition.partitionKey();
 +            Set<Index.Indexer> indexers = indexes.stream()
 +                                                 .map(index -> index.indexerFor(key,
 +                                                                                nowInSec,
 +                                                                                opGroup,
 +                                                                                IndexTransaction.Type.UPDATE))
 +                                                 .collect(Collectors.toSet());
 +
 +            indexers.forEach(Index.Indexer::begin);
 +
 +            try (RowIterator filtered = UnfilteredRowIterators.filter(partition, nowInSec))
 +            {
 +                if (!filtered.staticRow().isEmpty())
 +                    indexers.forEach(indexer -> indexer.insertRow(filtered.staticRow()));
 +
 +                while (filtered.hasNext())
 +                {
 +                    Row row = filtered.next();
 +                    indexers.forEach(indexer -> indexer.insertRow(row));
 +                }
 +            }
 +
 +            indexers.forEach(Index.Indexer::finish);
 +        }
 +    }
 +
 +    /**
 +     * Delete all data from all indexes for this partition.
 +     * For when cleanup rips a partition out entirely.
 +     *
 +     * TODO : improve cleanup transaction to batch updates & perform them async
 +     */
 +    public void deletePartition(UnfilteredRowIterator partition, int nowInSec)
 +    {
 +        // we need to acquire memtable lock because secondary index deletion may
 +        // cause a race (see CASSANDRA-3712). This is done internally by the
 +        // index transaction when it commits
 +        CleanupTransaction indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                                    partition.columns(),
 +                                                                    nowInSec);
 +        indexTransaction.start();
 +        indexTransaction.onPartitionDeletion(partition.partitionLevelDeletion());
 +        indexTransaction.commit();
 +
 +        while (partition.hasNext())
 +        {
 +            Unfiltered unfiltered = partition.next();
 +            if (unfiltered.kind() != Unfiltered.Kind.ROW)
 +                continue;
 +
 +            indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                     partition.columns(),
 +                                                     nowInSec);
 +            indexTransaction.start();
 +            indexTransaction.onRowDelete((Row)unfiltered);
 +            indexTransaction.commit();
 +        }
 +    }
 +
 +    /**
 +     * Called at query time to choose which (if any) of the registered index implementations to use for a given query.
 +     *
 +     * This is a two step processes, firstly compiling the set of searchable indexes then choosing the one which reduces
 +     * the search space the most.
 +     *
 +     * In the first phase, if the command's RowFilter contains any custom index expressions, the indexes that they
 +     * specify are automatically included. Following that, the registered indexes are filtered to include only those
 +     * which support the standard expressions in the RowFilter.
 +     *
 +     * The filtered set then sorted by selectivity, as reported by the Index implementations' getEstimatedResultRows
 +     * method.
 +     *
 +     * Implementation specific validation of the target expression, either custom or standard, by the selected
 +     * index should be performed in the searcherFor method to ensure that we pick the right index regardless of
 +     * the validity of the expression.
 +     *
 +     * This method is only called once during the lifecycle of a ReadCommand and the result is
 +     * cached for future use when obtaining a Searcher, getting the index's underlying CFS for
 +     * ReadOrderGroup, or an estimate of the result size from an average index query.
 +     *
 +     * @param command ReadCommand to be executed
 +     * @return an Index instance, ready to use during execution of the command, or null if none
 +     * of the registered indexes can support the command.
 +     */
 +    public Index getBestIndexFor(ReadCommand command)
 +    {
 +        if (indexes.isEmpty() || command.rowFilter().isEmpty())
 +            return null;
 +
 +        List<Index> searchableIndexes = new ArrayList<>();
 +        for (RowFilter.Expression expression : command.rowFilter())
 +        {
 +            if (expression.isCustom())
 +            {
 +                RowFilter.CustomExpression customExpression = (RowFilter.CustomExpression)expression;
 +                searchableIndexes.add(indexes.get(customExpression.getTargetIndex().name));
 +            }
 +            else
 +            {
 +                indexes.values().stream()
 +                       .filter(index -> index.supportsExpression(expression.column(), expression.operator()))
 +                       .forEach(searchableIndexes::add);
 +            }
 +        }
 +
 +        if (searchableIndexes.isEmpty())
 +        {
-             logger.debug("No applicable indexes found");
++            logger.trace("No applicable indexes found");
 +            Tracing.trace("No applicable indexes found");
 +            return null;
 +        }
 +
 +        Index selected = searchableIndexes.size() == 1
 +                         ? searchableIndexes.get(0)
 +                         : searchableIndexes.stream()
 +                                            .max((a, b) -> Longs.compare(a.getEstimatedResultRows(),
 +                                                                         b.getEstimatedResultRows()))
 +                                            .orElseThrow(() -> new AssertionError("Could not select most selective index"));
 +
 +        // pay for an additional threadlocal get() rather than build the strings unnecessarily
 +        if (Tracing.isTracing())
 +        {
 +            Tracing.trace("Index mean cardinalities are {}. Scanning with {}.",
 +                          searchableIndexes.stream().map(i -> i.getIndexName() + ':' + i.getEstimatedResultRows())
 +                                           .collect(Collectors.joining(",")),
 +                          selected.getIndexName());
 +        }
 +        return selected;
 +    }
 +
 +    /**
 +     * Called at write time to ensure that values present in the update
 +     * are valid according to the rules of all registered indexes which
 +     * will process it. The partition key as well as the clustering and
 +     * cell values for each row in the update may be checked by index
 +     * implementations
 +     * @param update PartitionUpdate containing the values to be validated by registered Index implementations
 +     * @throws InvalidRequestException
 +     */
 +    public void validate(PartitionUpdate update) throws InvalidRequestException
 +    {
 +        indexes.values()
 +               .stream()
 +               .filter(i -> i.indexes(update.columns()))
 +               .forEach(i -> i.validate(update));
 +    }
 +
 +    /**
 +     * IndexRegistry methods
 +     */
 +    public void registerIndex(Index index)
 +    {
 +        indexes.put(index.getIndexMetadata().name, index);
-         logger.debug("Registered index {}", index.getIndexMetadata().name);
++        logger.trace("Registered index {}", index.getIndexMetadata().name);
 +    }
 +
 +    public void unregisterIndex(Index index)
 +    {
 +        Index removed = indexes.remove(index.getIndexMetadata().name);
-         logger.debug(removed == null ? "Index {} was not registered" : "Removed index {} from registry",
++        logger.trace(removed == null ? "Index {} was not registered" : "Removed index {} from registry",
 +                     index.getIndexMetadata().name);
 +    }
 +
 +    public Index getIndex(IndexMetadata metadata)
 +    {
 +        return indexes.get(metadata.name);
 +    }
 +
 +    public Collection<Index> listIndexes()
 +    {
 +        return ImmutableSet.copyOf(indexes.values());
 +    }
 +
 +    /**
 +     * Handling of index updates.
 +     * Implementations of the various IndexTransaction interfaces, for keeping indexes in sync with base data
 +     * during updates, compaction and cleanup. Plus factory methods for obtaining transaction instances.
 +     */
 +
 +    /**
 +     * Transaction for updates on the write path.
 +     */
 +    public UpdateTransaction newUpdateTransaction(PartitionUpdate update, OpOrder.Group opGroup, int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return UpdateTransaction.NO_OP;
 +
 +        // todo : optimize lookup, we can probably cache quite a bit of stuff, rather than doing
 +        // a linear scan every time. Holding off that though until CASSANDRA-7771 to figure out
 +        // exactly how indexes are to be identified & associated with a given partition update
 +        Index.Indexer[] indexers = indexes.values().stream()
 +                                          .filter(i -> i.indexes(update.columns()))
 +                                          .map(i -> i.indexerFor(update.partitionKey(),
 +                                                                 nowInSec,
 +                                                                 opGroup,
 +                                                                 IndexTransaction.Type.UPDATE))
 +                                          .toArray(Index.Indexer[]::new);
 +
 +        return indexers.length == 0 ? UpdateTransaction.NO_OP : new WriteTimeTransaction(indexers);
 +    }
 +
 +    /**
 +     * Transaction for use when merging rows during compaction
 +     */
 +    public CompactionTransaction newCompactionTransaction(DecoratedKey key,
 +                                                          PartitionColumns partitionColumns,
 +                                                          int versions,
 +                                                          int nowInSec)
 +    {
 +        // the check for whether there are any registered indexes is already done in CompactionIterator
 +
 +        Index[] interestedIndexes = indexes.values().stream()
 +                                           .filter(i -> i.indexes(partitionColumns))
 +                                           .toArray(Index[]::new);
 +
 +        return interestedIndexes.length == 0
 +               ? CompactionTransaction.NO_OP
 +               : new IndexGCTransaction(key, versions, nowInSec, interestedIndexes);
 +    }
 +
 +    /**
 +     * Transaction for use when removing partitions during cleanup
 +     */
 +    public CleanupTransaction newCleanupTransaction(DecoratedKey key,
 +                                                    PartitionColumns partitionColumns,
 +                                                    int nowInSec)
 +    {
 +        //
 +        if (!hasIndexes())
 +            return CleanupTransaction.NO_OP;
 +
 +        Index[] interestedIndexes = indexes.values().stream()
 +                                           .filter(i -> i.indexes(partitionColumns))
 +                                           .toArray(Index[]::new);
 +
 +        return interestedIndexes.length == 0
 +               ? CleanupTransaction.NO_OP
 +               : new CleanupGCTransaction(key, nowInSec, interestedIndexes);
 +    }
 +
 +    /**
 +     * A single use transaction for processing a partition update on the regular write path
 +     */
 +    private static final class WriteTimeTransaction implements UpdateTransaction
 +    {
 +        private final Index.Indexer[] indexers;
 +
 +        private WriteTimeTransaction(Index.Indexer...indexers)
 +        {
 +            // don't allow null indexers, if we don't need any use a NullUpdater object
 +            for (Index.Indexer indexer : indexers) assert indexer != null;
 +            this.indexers = indexers;
 +        }
 +
 +        public void start()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.begin();
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.partitionDelete(deletionTime);
 +        }
 +
 +        public void onRangeTombstone(RangeTombstone tombstone)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.rangeTombstone(tombstone);
 +        }
 +
 +        public void onInserted(Row row)
 +        {
 +            Arrays.stream(indexers).forEach(h -> h.insertRow(row));
 +        }
 +
 +        public void onUpdated(Row existing, Row updated)
 +        {
 +            final Row.Builder toRemove = BTreeRow.sortedBuilder();
 +            toRemove.newRow(existing.clustering());
 +            final Row.Builder toInsert = BTreeRow.sortedBuilder();
 +            toInsert.newRow(updated.clustering());
 +            // diff listener collates the columns to be added & removed from the indexes
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                    if (merged != null && merged != original)
 +                        toInsert.addPrimaryKeyLivenessInfo(merged);
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (merged != null && merged != original)
 +                        toInsert.addCell(merged);
 +
 +                    if (merged == null || (original != null && shouldCleanupOldValue(original, merged)))
 +                        toRemove.addCell(original);
 +
 +                }
 +            };
 +            Rows.diff(diffListener, updated, existing);
 +            Row oldRow = toRemove.build();
 +            Row newRow = toInsert.build();
 +            for (Index.Indexer indexer : indexers)
 +                indexer.updateRow(oldRow, newRow);
 +        }
 +
 +        public void commit()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.finish();
 +        }
 +
 +        private boolean shouldCleanupOldValue(Cell oldCell, Cell newCell)
 +        {
 +            // If either the value or timestamp is different, then we
 +            // should delete from the index. If not, then we can infer that
 +            // at least one of the cells is an ExpiringColumn and that the
 +            // difference is in the expiry time. In this case, we don't want to
 +            // delete the old value from the index as the tombstone we insert
 +            // will just hide the inserted value.
 +            // Completely identical cells (including expiring columns with
 +            // identical ttl & localExpirationTime) will not get this far due
 +            // to the oldCell.equals(newCell) in StandardUpdater.update
 +            return !oldCell.value().equals(newCell.value()) || oldCell.timestamp() != newCell.timestamp();
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during compaction where the only
 +     * operation is to merge rows
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class IndexGCTransaction implements CompactionTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final int versions;
 +        private final int nowInSec;
 +        private final Index[] indexes;
 +
 +        private Row[] rows;
 +
 +        private IndexGCTransaction(DecoratedKey key,
 +                                   int versions,
 +                                   int nowInSec,
 +                                   Index...indexes)
 +        {
 +            // don't allow null indexers, if we don't have any, use a noop transaction
 +            for (Index index : indexes) assert index != null;
 +
 +            this.key = key;
 +            this.versions = versions;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +            if (versions > 0)
 +                rows = new Row[versions];
 +        }
 +
 +        public void onRowMerge(Row merged, Row...versions)
 +        {
 +            // Diff listener constructs rows representing deltas between the merged and original versions
 +            // These delta rows are then passed to registered indexes for removal processing
 +            final Row.Builder[] builders = new Row.Builder[versions.length];
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (original != null && merged == null)
 +                    {
 +                        if (builders[i] == null)
 +                        {
 +                            builders[i] = BTreeRow.sortedBuilder();
 +                            builders[i].newRow(clustering);
 +                        }
 +                        builders[i].addCell(original);
 +                    }
 +                }
 +            };
 +
 +            Rows.diff(diffListener, merged, versions);
 +
 +            for(int i = 0; i < builders.length; i++)
 +                if (builders[i] != null)
 +                    rows[i] = builders[i].build();
 +        }
 +
 +        public void commit()
 +        {
 +            if (rows == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, nowInSec, opGroup, Type.COMPACTION);
 +                    indexer.begin();
 +                    for (Row row : rows)
 +                        if (row != null)
 +                            indexer.removeRow(row);
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during cleanup, where
 +     * partitions and rows are only removed
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class CleanupGCTransaction implements CleanupTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final int nowInSec;
 +        private final Index[] indexes;
 +
 +        private Row row;
 +        private DeletionTime partitionDelete;
 +
 +        private CleanupGCTransaction(DecoratedKey key,
 +                                     int nowInSec,
 +                                     Index...indexes)
 +        {
 +            // don't allow null indexers, if we don't have any, use a noop transaction
 +            for (Index index : indexes) assert index != null;
 +
 +            this.key = key;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            partitionDelete = deletionTime;
 +        }
 +
 +        public void onRowDelete(Row row)
 +        {
 +            this.row = row;
 +        }
 +
 +        public void commit()
 +        {
 +            if (row == null && partitionDelete == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, nowInSec, opGroup, Type.CLEANUP);
 +                    indexer.begin();
 +                    if (row != null)
 +                        indexer.removeRow(row);
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    private static void executeBlocking(Callable<?> task)
 +    {
 +        if (null != task)
 +            FBUtilities.waitOnFuture(blockingExecutor.submit(task));
 +    }
 +
 +    private static void executeAllBlocking(Stream<Index> indexers, Function<Index, Callable<?>> function)
 +    {
 +        List<Future<?>> waitFor = new ArrayList<>();
 +        indexers.forEach(indexer -> {
 +            Callable<?> task = function.apply(indexer);
 +            if (null != task)
 +                waitFor.add(blockingExecutor.submit(task));
 +        });
 +        FBUtilities.waitOnFutures(waitFor);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/index/internal/CassandraIndex.java
index f6a10e5,0000000..93f5d61
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
@@@ -1,826 -1,0 +1,826 @@@
 +package org.apache.cassandra.index.internal;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.concurrent.Callable;
 +import java.util.concurrent.Future;
 +import java.util.function.BiFunction;
 +import java.util.regex.Matcher;
 +import java.util.regex.Pattern;
 +import java.util.stream.Collectors;
 +import java.util.stream.StreamSupport;
 +
 +import org.apache.commons.lang3.StringUtils;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.cql3.Operator;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CollectionType;
 +import org.apache.cassandra.db.partitions.PartitionIterator;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.dht.LocalPartitioner;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.index.IndexRegistry;
 +import org.apache.cassandra.index.SecondaryIndexBuilder;
 +import org.apache.cassandra.index.internal.composites.CompositesSearcher;
 +import org.apache.cassandra.index.internal.keys.KeysSearcher;
 +import org.apache.cassandra.index.transactions.IndexTransaction;
 +import org.apache.cassandra.index.transactions.UpdateTransaction;
 +import org.apache.cassandra.io.sstable.ReducingKeyIterator;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.Pair;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.concurrent.Refs;
 +
 +/**
 + * Index implementation which indexes the values for a single column in the base
 + * table and which stores its index data in a local, hidden table.
 + */
 +public abstract class CassandraIndex implements Index
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(CassandraIndex.class);
 +
 +    public static final Pattern TARGET_REGEX = Pattern.compile("^(keys|entries|values|full)\\((.+)\\)$");
 +
 +    public final ColumnFamilyStore baseCfs;
 +    protected IndexMetadata metadata;
 +    protected ColumnFamilyStore indexCfs;
 +    protected ColumnDefinition indexedColumn;
 +    protected CassandraIndexFunctions functions;
 +
 +    protected CassandraIndex(ColumnFamilyStore baseCfs, IndexMetadata indexDef)
 +    {
 +        this.baseCfs = baseCfs;
 +        setMetadata(indexDef);
 +    }
 +
 +    /**
 +     * Returns true if an index of this type can support search predicates of the form [column] OPERATOR [value]
 +     * @param indexedColumn
 +     * @param operator
 +     * @return
 +     */
 +    protected boolean supportsOperator(ColumnDefinition indexedColumn, Operator operator)
 +    {
 +        return operator == Operator.EQ;
 +    }
 +
 +    /**
 +     * Used to construct an the clustering for an entry in the index table based on values from the base data.
 +     * The clustering columns in the index table encode the values required to retrieve the correct data from the base
 +     * table and varies depending on the kind of the indexed column. See indexCfsMetadata for more details
 +     * Used whenever a row in the index table is written or deleted.
 +     * @param partitionKey from the base data being indexed
 +     * @param prefix from the base data being indexed
 +     * @param path from the base data being indexed
 +     * @return a clustering prefix to be used to insert into the index table
 +     */
 +    protected abstract CBuilder buildIndexClusteringPrefix(ByteBuffer partitionKey,
 +                                                           ClusteringPrefix prefix,
 +                                                           CellPath path);
 +
 +    /**
 +     * Used at search time to convert a row in the index table into a simple struct containing the values required
 +     * to retrieve the corresponding row from the base table.
 +     * @param indexedValue the partition key of the indexed table (i.e. the value that was indexed)
 +     * @param indexEntry a row from the index table
 +     * @return
 +     */
 +    public abstract IndexEntry decodeEntry(DecoratedKey indexedValue,
 +                                           Row indexEntry);
 +
 +    /**
 +     * Check whether a value retrieved from an index is still valid by comparing it to current row from the base table.
 +     * Used at read time to identify out of date index entries so that they can be excluded from search results and
 +     * repaired
 +     * @param row the current row from the primary data table
 +     * @param indexValue the value we retrieved from the index
 +     * @param nowInSec
 +     * @return true if the index is out of date and the entry should be dropped
 +     */
 +    public abstract boolean isStale(Row row, ByteBuffer indexValue, int nowInSec);
 +
 +    /**
 +     * Extract the value to be inserted into the index from the components of the base data
 +     * @param partitionKey from the primary data
 +     * @param clustering from the primary data
 +     * @param path from the primary data
 +     * @param cellValue from the primary data
 +     * @return a ByteBuffer containing the value to be inserted in the index. This will be used to make the partition
 +     * key in the index table
 +     */
 +    protected abstract ByteBuffer getIndexedValue(ByteBuffer partitionKey,
 +                                                  Clustering clustering,
 +                                                  CellPath path,
 +                                                  ByteBuffer cellValue);
 +
 +    public ColumnDefinition getIndexedColumn()
 +    {
 +        return indexedColumn;
 +    }
 +
 +    public ClusteringComparator getIndexComparator()
 +    {
 +        return indexCfs.metadata.comparator;
 +    }
 +
 +    public ColumnFamilyStore getIndexCfs()
 +    {
 +        return indexCfs;
 +    }
 +
 +    public void register(IndexRegistry registry)
 +    {
 +        registry.registerIndex(this);
 +    }
 +
 +    public Callable<?> getInitializationTask()
 +    {
 +        // if we're just linking in the index on an already-built index post-restart
 +        // we've nothing to do. Otherwise, submit for building via SecondaryIndexBuilder
 +        return isBuilt() ? null : getBuildIndexTask();
 +    }
 +
 +    public IndexMetadata getIndexMetadata()
 +    {
 +        return metadata;
 +    }
 +
 +    public String getIndexName()
 +    {
 +        return metadata.name;
 +    }
 +
 +    public Optional<ColumnFamilyStore> getBackingTable()
 +    {
 +        return indexCfs == null ? Optional.empty() : Optional.of(indexCfs);
 +    }
 +
 +    public Callable<Void> getBlockingFlushTask()
 +    {
 +        return () -> {
 +            indexCfs.forceBlockingFlush();
 +            return null;
 +        };
 +    }
 +
 +    public Callable<?> getInvalidateTask()
 +    {
 +        return () -> {
 +            markRemoved();
 +            invalidate();
 +            return null;
 +        };
 +    }
 +
 +    public Callable<?> getMetadataReloadTask(IndexMetadata indexDef)
 +    {
 +        setMetadata(indexDef);
 +        return () -> {
 +            indexCfs.metadata.reloadIndexMetadataProperties(baseCfs.metadata);
 +            indexCfs.reload();
 +            return null;
 +        };
 +    }
 +
 +    private void setMetadata(IndexMetadata indexDef)
 +    {
 +        metadata = indexDef;
 +        Pair<ColumnDefinition, IndexTarget.Type> target = parseTarget(baseCfs.metadata, indexDef);
 +        functions = getFunctions(indexDef, target);
 +        CFMetaData cfm = indexCfsMetadata(baseCfs.metadata, indexDef);
 +        indexCfs = ColumnFamilyStore.createColumnFamilyStore(baseCfs.keyspace,
 +                                                             cfm.cfName,
 +                                                             cfm,
 +                                                             baseCfs.getTracker().loadsstables);
 +        indexedColumn = target.left;
 +    }
 +
 +    public Callable<?> getTruncateTask(final long truncatedAt)
 +    {
 +        return () -> {
 +            indexCfs.discardSSTables(truncatedAt);
 +            return null;
 +        };
 +    }
 +
 +    public boolean shouldBuildBlocking()
 +    {
 +        // built-in indexes are always included in builds initiated from SecondaryIndexManager
 +        return true;
 +    }
 +
 +    public boolean indexes(PartitionColumns columns)
 +    {
 +        // if we have indexes on the partition key or clustering columns, return true
 +        return isPrimaryKeyIndex() || columns.contains(indexedColumn);
 +    }
 +
 +    public boolean dependsOn(ColumnDefinition column)
 +    {
 +        return indexedColumn.name.equals(column.name);
 +    }
 +
 +    public boolean supportsExpression(ColumnDefinition column, Operator operator)
 +    {
 +        return indexedColumn.name.equals(column.name)
 +               && supportsOperator(indexedColumn, operator);
 +    }
 +
 +    private boolean supportsExpression(RowFilter.Expression expression)
 +    {
 +        return supportsExpression(expression.column(), expression.operator());
 +    }
 +
 +    public AbstractType<?> customExpressionValueType()
 +    {
 +        return null;
 +    }
 +
 +    public long getEstimatedResultRows()
 +    {
 +        return indexCfs.getMeanColumns();
 +    }
 +
 +    /**
 +     * No post processing of query results, just return them unchanged
 +     */
 +    public BiFunction<PartitionIterator, ReadCommand, PartitionIterator> postProcessorFor(ReadCommand command)
 +    {
 +        return (partitionIterator, readCommand) -> partitionIterator;
 +    }
 +
 +    public RowFilter getPostIndexQueryFilter(RowFilter filter)
 +    {
 +        return getTargetExpression(filter.getExpressions()).map(filter::without)
 +                                                           .orElse(filter);
 +    }
 +
 +    private Optional<RowFilter.Expression> getTargetExpression(List<RowFilter.Expression> expressions)
 +    {
 +        return expressions.stream().filter(this::supportsExpression).findFirst();
 +    }
 +
 +    public Index.Searcher searcherFor(ReadCommand command)
 +    {
 +        Optional<RowFilter.Expression> target = getTargetExpression(command.rowFilter().getExpressions());
 +
 +        if (target.isPresent())
 +        {
 +            target.get().validateForIndexing();
 +            switch (getIndexMetadata().kind)
 +            {
 +                case COMPOSITES:
 +                    return new CompositesSearcher(command, target.get(), this);
 +                case KEYS:
 +                    return new KeysSearcher(command, target.get(), this);
 +                default:
 +                    throw new IllegalStateException(String.format("Unsupported index type %s for index %s on %s",
 +                                                                  metadata.kind,
 +                                                                  metadata.name,
 +                                                                  indexedColumn.name.toString()));
 +            }
 +        }
 +
 +        return null;
 +
 +    }
 +
 +    public void validate(PartitionUpdate update) throws InvalidRequestException
 +    {
 +        switch (indexedColumn.kind)
 +        {
 +            case PARTITION_KEY:
 +                validatePartitionKey(update.partitionKey());
 +                break;
 +            case CLUSTERING:
 +                validateClusterings(update);
 +                break;
 +            case REGULAR:
 +                validateRows(update);
 +                break;
 +            case STATIC:
 +                validateRows(Collections.singleton(update.staticRow()));
 +                break;
 +        }
 +    }
 +
 +    public Indexer indexerFor(final DecoratedKey key,
 +                              final int nowInSec,
 +                              final OpOrder.Group opGroup,
 +                              final IndexTransaction.Type transactionType)
 +    {
 +        return new Indexer()
 +        {
 +            public void begin()
 +            {
 +            }
 +
 +            public void partitionDelete(DeletionTime deletionTime)
 +            {
 +            }
 +
 +            public void rangeTombstone(RangeTombstone tombstone)
 +            {
 +            }
 +
 +            public void insertRow(Row row)
 +            {
 +                if (isPrimaryKeyIndex())
 +                {
 +                    indexPrimaryKey(row.clustering(),
 +                                    getPrimaryKeyIndexLiveness(row),
 +                                    row.deletion());
 +                }
 +                else
 +                {
 +                    if (indexedColumn.isComplex())
 +                        indexCells(row.clustering(), row.getComplexColumnData(indexedColumn));
 +                    else
 +                        indexCell(row.clustering(), row.getCell(indexedColumn));
 +                }
 +            }
 +
 +            public void removeRow(Row row)
 +            {
 +                if (isPrimaryKeyIndex())
 +                    indexPrimaryKey(row.clustering(), row.primaryKeyLivenessInfo(), row.deletion());
 +
 +                if (indexedColumn.isComplex())
 +                    removeCells(row.clustering(), row.getComplexColumnData(indexedColumn));
 +                else
 +                    removeCell(row.clustering(), row.getCell(indexedColumn));
 +            }
 +
 +
 +            public void updateRow(Row oldRow, Row newRow)
 +            {
 +                if (isPrimaryKeyIndex())
 +                    indexPrimaryKey(newRow.clustering(),
 +                                    newRow.primaryKeyLivenessInfo(),
 +                                    newRow.deletion());
 +
 +                if (indexedColumn.isComplex())
 +                {
 +                    indexCells(newRow.clustering(), newRow.getComplexColumnData(indexedColumn));
 +                    removeCells(oldRow.clustering(), oldRow.getComplexColumnData(indexedColumn));
 +                }
 +                else
 +                {
 +                    indexCell(newRow.clustering(), newRow.getCell(indexedColumn));
 +                    removeCell(oldRow.clustering(), oldRow.getCell(indexedColumn));
 +                }
 +            }
 +
 +            public void finish()
 +            {
 +            }
 +
 +            private void indexCells(Clustering clustering, Iterable<Cell> cells)
 +            {
 +                if (cells == null)
 +                    return;
 +
 +                for (Cell cell : cells)
 +                    indexCell(clustering, cell);
 +            }
 +
 +            private void indexCell(Clustering clustering, Cell cell)
 +            {
 +                if (cell == null || !cell.isLive(nowInSec))
 +                    return;
 +
 +                insert(key.getKey(),
 +                       clustering,
 +                       cell,
 +                       LivenessInfo.create(cell.timestamp(), cell.ttl(), cell.localDeletionTime()),
 +                       opGroup);
 +            }
 +
 +            private void removeCells(Clustering clustering, Iterable<Cell> cells)
 +            {
 +                if (cells == null)
 +                    return;
 +
 +                for (Cell cell : cells)
 +                    removeCell(clustering, cell);
 +            }
 +
 +            private void removeCell(Clustering clustering, Cell cell)
 +            {
 +                if (cell == null || !cell.isLive(nowInSec))
 +                    return;
 +
 +                delete(key.getKey(), clustering, cell, opGroup, nowInSec);
 +            }
 +
 +            private void indexPrimaryKey(final Clustering clustering,
 +                                         final LivenessInfo liveness,
 +                                         final Row.Deletion deletion)
 +            {
 +                if (liveness.timestamp() != LivenessInfo.NO_TIMESTAMP)
 +                    insert(key.getKey(), clustering, null, liveness, opGroup);
 +
 +                if (!deletion.isLive())
 +                    delete(key.getKey(), clustering, deletion.time(), opGroup);
 +            }
 +
 +            private LivenessInfo getPrimaryKeyIndexLiveness(Row row)
 +            {
 +                long timestamp = row.primaryKeyLivenessInfo().timestamp();
 +                int ttl = row.primaryKeyLivenessInfo().ttl();
 +                for (Cell cell : row.cells())
 +                {
 +                    long cellTimestamp = cell.timestamp();
 +                    if (cell.isLive(nowInSec))
 +                    {
 +                        if (cellTimestamp > timestamp)
 +                        {
 +                            timestamp = cellTimestamp;
 +                            ttl = cell.ttl();
 +                        }
 +                    }
 +                }
 +                return LivenessInfo.create(baseCfs.metadata, timestamp, ttl, nowInSec);
 +            }
 +        };
 +    }
 +
 +    /**
 +     * Specific to internal indexes, this is called by a
 +     * searcher when it encounters a stale entry in the index
 +     * @param indexKey the partition key in the index table
 +     * @param indexClustering the clustering in the index table
 +     * @param deletion deletion timestamp etc
 +     * @param opGroup the operation under which to perform the deletion
 +     */
 +    public void deleteStaleEntry(DecoratedKey indexKey,
 +                                 Clustering indexClustering,
 +                                 DeletionTime deletion,
 +                                 OpOrder.Group opGroup)
 +    {
 +        doDelete(indexKey, indexClustering, deletion, opGroup);
-         logger.debug("Removed index entry for stale value {}", indexKey);
++        logger.trace("Removed index entry for stale value {}", indexKey);
 +    }
 +
 +    /**
 +     * Called when adding a new entry to the index
 +     */
 +    private void insert(ByteBuffer rowKey,
 +                        Clustering clustering,
 +                        Cell cell,
 +                        LivenessInfo info,
 +                        OpOrder.Group opGroup)
 +    {
 +        DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey,
 +                                                               clustering,
 +                                                               cell));
 +        Row row = BTreeRow.noCellLiveRow(buildIndexClustering(rowKey, clustering, cell), info);
 +        PartitionUpdate upd = partitionUpdate(valueKey, row);
 +        indexCfs.apply(upd, UpdateTransaction.NO_OP, opGroup, null);
-         logger.debug("Inserted entry into index for value {}", valueKey);
++        logger.trace("Inserted entry into index for value {}", valueKey);
 +    }
 +
 +    /**
 +     * Called when deleting entries on non-primary key columns
 +     */
 +    private void delete(ByteBuffer rowKey,
 +                        Clustering clustering,
 +                        Cell cell,
 +                        OpOrder.Group opGroup,
 +                        int nowInSec)
 +    {
 +        DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey,
 +                                                               clustering,
 +                                                               cell));
 +        doDelete(valueKey,
 +                 buildIndexClustering(rowKey, clustering, cell),
 +                 new DeletionTime(cell.timestamp(), nowInSec),
 +                 opGroup);
 +    }
 +
 +    /**
 +     * Called when deleting entries from indexes on primary key columns
 +     */
 +    private void delete(ByteBuffer rowKey,
 +                        Clustering clustering,
 +                        DeletionTime deletion,
 +                        OpOrder.Group opGroup)
 +    {
 +        DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey,
 +                                                               clustering,
 +                                                               null));
 +        doDelete(valueKey,
 +                 buildIndexClustering(rowKey, clustering, null),
 +                 deletion,
 +                 opGroup);
 +    }
 +
 +    private void doDelete(DecoratedKey indexKey,
 +                          Clustering indexClustering,
 +                          DeletionTime deletion,
 +                          OpOrder.Group opGroup)
 +    {
 +        Row row = BTreeRow.emptyDeletedRow(indexClustering, Row.Deletion.regular(deletion));
 +        PartitionUpdate upd = partitionUpdate(indexKey, row);
 +        indexCfs.apply(upd, UpdateTransaction.NO_OP, opGroup, null);
-         logger.debug("Removed index entry for value {}", indexKey);
++        logger.trace("Removed index entry for value {}", indexKey);
 +    }
 +
 +    private void validatePartitionKey(DecoratedKey partitionKey) throws InvalidRequestException
 +    {
 +        assert indexedColumn.isPartitionKey();
 +        validateIndexedValue(getIndexedValue(partitionKey.getKey(), null, null));
 +    }
 +
 +    private void validateClusterings(PartitionUpdate update) throws InvalidRequestException
 +    {
 +        assert indexedColumn.isClusteringColumn();
 +        for (Row row : update)
 +            validateIndexedValue(getIndexedValue(null, row.clustering(), null));
 +    }
 +
 +    private void validateRows(Iterable<Row> rows)
 +    {
 +        assert !indexedColumn.isPrimaryKeyColumn();
 +        for (Row row : rows)
 +        {
 +            if (indexedColumn.isComplex())
 +            {
 +                ComplexColumnData data = row.getComplexColumnData(indexedColumn);
 +                if (data != null)
 +                {
 +                    for (Cell cell : data)
 +                    {
 +                        validateIndexedValue(getIndexedValue(null, null, cell.path(), cell.value()));
 +                    }
 +                }
 +            }
 +            else
 +            {
 +                validateIndexedValue(getIndexedValue(null, null, row.getCell(indexedColumn)));
 +            }
 +        }
 +    }
 +
 +    private void validateIndexedValue(ByteBuffer value)
 +    {
 +        if (value != null && value.remaining() >= FBUtilities.MAX_UNSIGNED_SHORT)
 +            throw new InvalidRequestException(String.format(
 +                                                           "Cannot index value of size %d for index %s on %s.%s(%s) (maximum allowed size=%d)",
 +                                                           value.remaining(),
 +                                                           getIndexName(),
 +                                                           baseCfs.metadata.ksName,
 +                                                           baseCfs.metadata.cfName,
 +                                                           indexedColumn.name.toString(),
 +                                                           FBUtilities.MAX_UNSIGNED_SHORT));
 +    }
 +
 +    private ByteBuffer getIndexedValue(ByteBuffer rowKey,
 +                                       Clustering clustering,
 +                                       Cell cell)
 +    {
 +        return getIndexedValue(rowKey,
 +                               clustering,
 +                               cell == null ? null : cell.path(),
 +                               cell == null ? null : cell.value()
 +        );
 +    }
 +
 +    private Clustering buildIndexClustering(ByteBuffer rowKey,
 +                                            Clustering clustering,
 +                                            Cell cell)
 +    {
 +        return buildIndexClusteringPrefix(rowKey,
 +                                          clustering,
 +                                          cell == null ? null : cell.path()).build();
 +    }
 +
 +    private DecoratedKey getIndexKeyFor(ByteBuffer value)
 +    {
 +        return indexCfs.decorateKey(value);
 +    }
 +
 +    private PartitionUpdate partitionUpdate(DecoratedKey valueKey, Row row)
 +    {
 +        return PartitionUpdate.singleRowUpdate(indexCfs.metadata, valueKey, row);
 +    }
 +
 +    private void invalidate()
 +    {
 +        // interrupt in-progress compactions
 +        Collection<ColumnFamilyStore> cfss = Collections.singleton(indexCfs);
 +        CompactionManager.instance.interruptCompactionForCFs(cfss, true);
 +        CompactionManager.instance.waitForCessation(cfss);
 +        Keyspace.writeOrder.awaitNewBarrier();
 +        indexCfs.forceBlockingFlush();
 +        indexCfs.readOrdering.awaitNewBarrier();
 +        indexCfs.invalidate();
 +    }
 +
 +    private boolean isBuilt()
 +    {
 +        return SystemKeyspace.isIndexBuilt(baseCfs.keyspace.getName(), getIndexName());
 +    }
 +
 +    private void markBuilt()
 +    {
 +        SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), getIndexName());
 +    }
 +
 +    private void markRemoved()
 +    {
 +        SystemKeyspace.setIndexRemoved(baseCfs.keyspace.getName(), getIndexName());
 +    }
 +
 +    private boolean isPrimaryKeyIndex()
 +    {
 +        return indexedColumn.isPrimaryKeyColumn();
 +    }
 +
 +    private Callable<?> getBuildIndexTask()
 +    {
 +        return () -> {
 +            buildBlocking();
 +            return null;
 +        };
 +    }
 +
 +    private void buildBlocking()
 +    {
 +        baseCfs.forceBlockingFlush();
 +
 +        try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.select(SSTableSet.CANONICAL));
 +             Refs<SSTableReader> sstables = viewFragment.refs)
 +        {
 +            if (sstables.isEmpty())
 +            {
 +                logger.info("No SSTable data for {}.{} to build index {} from, marking empty index as built",
 +                            baseCfs.metadata.ksName,
 +                            baseCfs.metadata.cfName,
 +                            getIndexName());
 +                markBuilt();
 +                return;
 +            }
 +
 +            logger.info("Submitting index build of {} for data in {}",
 +                        getIndexName(),
 +                        getSSTableNames(sstables));
 +
 +            SecondaryIndexBuilder builder = new SecondaryIndexBuilder(baseCfs,
 +                                                                      Collections.singleton(this),
 +                                                                      new ReducingKeyIterator(sstables));
 +            Future<?> future = CompactionManager.instance.submitIndexBuild(builder);
 +            FBUtilities.waitOnFuture(future);
 +            indexCfs.forceBlockingFlush();
 +            markBuilt();
 +        }
 +        logger.info("Index build of {} complete", getIndexName());
 +    }
 +
 +    private static String getSSTableNames(Collection<SSTableReader> sstables)
 +    {
 +        return StreamSupport.stream(sstables.spliterator(), false)
 +                            .map(SSTableReader::toString)
 +                            .collect(Collectors.joining(", "));
 +    }
 +
 +    /**
 +     * Construct the CFMetadata for an index table, the clustering columns in the index table
 +     * vary dependent on the kind of the indexed value.
 +     * @param baseCfsMetadata
 +     * @param indexMetadata
 +     * @return
 +     */
 +    public static final CFMetaData indexCfsMetadata(CFMetaData baseCfsMetadata, IndexMetadata indexMetadata)
 +    {
 +        Pair<ColumnDefinition, IndexTarget.Type> target = parseTarget(baseCfsMetadata, indexMetadata);
 +        CassandraIndexFunctions utils = getFunctions(indexMetadata, target);
 +        ColumnDefinition indexedColumn = target.left;
 +        AbstractType<?> indexedValueType = utils.getIndexedValueType(indexedColumn);
 +        CFMetaData.Builder builder = CFMetaData.Builder.create(baseCfsMetadata.ksName,
 +                                                               baseCfsMetadata.indexColumnFamilyName(indexMetadata))
 +                                                       .withId(baseCfsMetadata.cfId)
 +                                                       .withPartitioner(new LocalPartitioner(indexedValueType))
 +                                                       .addPartitionKey(indexedColumn.name, indexedColumn.type);
 +
 +        builder.addClusteringColumn("partition_key", baseCfsMetadata.partitioner.partitionOrdering());
 +        builder = utils.addIndexClusteringColumns(builder, baseCfsMetadata, indexedColumn);
 +        return builder.build().reloadIndexMetadataProperties(baseCfsMetadata);
 +    }
 +
 +    /**
 +     * Factory method for new CassandraIndex instances
 +     * @param baseCfs
 +     * @param indexMetadata
 +     * @return
 +     */
 +    public static CassandraIndex newIndex(ColumnFamilyStore baseCfs, IndexMetadata indexMetadata)
 +    {
 +        return getFunctions(indexMetadata, parseTarget(baseCfs.metadata, indexMetadata)).newIndexInstance(baseCfs, indexMetadata);
 +    }
 +
 +    // Public because it's also used to convert index metadata into a thrift-compatible format
 +    public static Pair<ColumnDefinition, IndexTarget.Type> parseTarget(CFMetaData cfm,
 +                                                                       IndexMetadata indexDef)
 +    {
 +        String target = indexDef.options.get("target");
 +        assert target != null : String.format("No target definition found for index %s", indexDef.name);
 +
 +        // if the regex matches then the target is in the form "keys(foo)", "entries(bar)" etc
 +        // if not, then it must be a simple column name and implictly its type is VALUES
 +        Matcher matcher = TARGET_REGEX.matcher(target);
 +        String columnName;
 +        IndexTarget.Type targetType;
 +        if (matcher.matches())
 +        {
 +            targetType = IndexTarget.Type.fromString(matcher.group(1));
 +            columnName = matcher.group(2);
 +        }
 +        else
 +        {
 +            columnName = target;
 +            targetType = IndexTarget.Type.VALUES;
 +        }
 +
 +        // in the case of a quoted column name the name in the target string
 +        // will be enclosed in quotes, which we need to unwrap. It may also
 +        // include quote characters internally, escaped like so:
 +        //      abc"def -> abc""def.
 +        // Because the target string is stored in a CQL compatible form, we
 +        // need to un-escape any such quotes to get the actual column name
 +        if (columnName.startsWith("\""))
 +        {
 +            columnName = StringUtils.substring(StringUtils.substring(columnName, 1), 0, -1);
 +            columnName = columnName.replaceAll("\"\"", "\"");
 +        }
 +
 +        // if it's not a CQL table, we can't assume that the column name is utf8, so
 +        // in that case we have to do a linear scan of the cfm's columns to get the matching one
 +        if (cfm.isCQLTable())
 +            return Pair.create(cfm.getColumnDefinition(new ColumnIdentifier(columnName, true)), targetType);
 +        else
 +            for (ColumnDefinition column : cfm.allColumns())
 +                if (column.name.toString().equals(columnName))
 +                    return Pair.create(column, targetType);
 +
 +        throw new RuntimeException(String.format("Unable to parse targets for index %s (%s)", indexDef.name, target));
 +    }
 +
 +    static CassandraIndexFunctions getFunctions(IndexMetadata indexDef,
 +                                                Pair<ColumnDefinition, IndexTarget.Type> target)
 +    {
 +        if (indexDef.isKeys())
 +            return CassandraIndexFunctions.KEYS_INDEX_FUNCTIONS;
 +
 +        ColumnDefinition indexedColumn = target.left;
 +        if (indexedColumn.type.isCollection() && indexedColumn.type.isMultiCell())
 +        {
 +            switch (((CollectionType)indexedColumn.type).kind)
 +            {
 +                case LIST:
 +                    return CassandraIndexFunctions.COLLECTION_VALUE_INDEX_FUNCTIONS;
 +                case SET:
 +                    return CassandraIndexFunctions.COLLECTION_KEY_INDEX_FUNCTIONS;
 +                case MAP:
 +                    switch (target.right)
 +                    {
 +                        case KEYS:
 +                            return CassandraIndexFunctions.COLLECTION_KEY_INDEX_FUNCTIONS;
 +                        case KEYS_AND_VALUES:
 +                            return CassandraIndexFunctions.COLLECTION_ENTRY_INDEX_FUNCTIONS;
 +                        case VALUES:
 +                            return CassandraIndexFunctions.COLLECTION_VALUE_INDEX_FUNCTIONS;
 +                    }
 +                    throw new AssertionError();
 +            }
 +        }
 +
 +        switch (indexedColumn.kind)
 +        {
 +            case CLUSTERING:
 +                return CassandraIndexFunctions.CLUSTERING_COLUMN_INDEX_FUNCTIONS;
 +            case REGULAR:
 +                return CassandraIndexFunctions.REGULAR_COLUMN_INDEX_FUNCTIONS;
 +            case PARTITION_KEY:
 +                return CassandraIndexFunctions.PARTITION_KEY_INDEX_FUNCTIONS;
 +            //case COMPACT_VALUE:
 +            //    return new CompositesIndexOnCompactValue();
 +        }
 +        throw new AssertionError();
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTable.java
index d66638e,b0aa89e..923ef82
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@@ -111,11 -113,9 +111,11 @@@ public abstract class SSTabl
  
              FileUtils.deleteWithConfirm(desc.filenameFor(component));
          }
 -        FileUtils.delete(desc.filenameFor(Component.SUMMARY));
 +
 +        if (components.contains(Component.SUMMARY))
 +            FileUtils.delete(desc.filenameFor(Component.SUMMARY));
  
-         logger.debug("Deleted {}", desc);
+         logger.trace("Deleted {}", desc);
          return true;
      }
  


[3/9] cassandra git commit: Add a separate production debug log for troubleshooting

Posted by bl...@apache.org.
Add a separate production debug log for troubleshooting

patch by Paulo Motta; reviewed by Ariel Weisberg  for CASSANDRA-10241


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

Branch: refs/heads/trunk
Commit: 4a849efeb7c7c1a54bc12094d2d6a9f3f008a2fa
Parents: f3ad68c
Author: Paulo Motta <pa...@gmail.com>
Authored: Tue Sep 22 22:00:23 2015 +0200
Committer: blerer <be...@datastax.com>
Committed: Tue Sep 22 22:00:23 2015 +0200

----------------------------------------------------------------------
 NEWS.txt                                        | 10 ++++
 conf/logback.xml                                | 55 ++++++++++++++++---
 .../cassandra/auth/CassandraAuthorizer.java     |  2 +-
 .../cassandra/auth/CassandraRoleManager.java    |  4 +-
 .../cassandra/auth/PasswordAuthenticator.java   |  4 +-
 .../apache/cassandra/auth/PermissionsCache.java |  2 +-
 .../org/apache/cassandra/auth/RolesCache.java   |  2 +-
 .../apache/cassandra/cache/AutoSavingCache.java |  8 +--
 .../cassandra/cache/SerializingCache.java       |  2 +-
 .../org/apache/cassandra/client/RingCache.java  |  2 +-
 .../DebuggableScheduledThreadPoolExecutor.java  |  2 +-
 .../DebuggableThreadPoolExecutor.java           |  2 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |  6 +--
 .../cql3/functions/JavaSourceUDFFactory.java    |  4 +-
 .../cql3/functions/ScriptBasedUDF.java          |  2 +-
 .../cql3/statements/CreateIndexStatement.java   |  2 +-
 .../apache/cassandra/db/BatchlogManager.java    | 10 ++--
 .../apache/cassandra/db/ColumnFamilyStore.java  | 50 ++++++++---------
 .../apache/cassandra/db/ConsistencyLevel.java   |  6 +--
 .../db/CounterMutationVerbHandler.java          |  2 +-
 .../db/DefinitionsUpdateVerbHandler.java        |  2 +-
 .../org/apache/cassandra/db/Directories.java    |  8 +--
 .../cassandra/db/HintedHandOffManager.java      | 24 ++++-----
 src/java/org/apache/cassandra/db/Keyspace.java  |  6 +--
 src/java/org/apache/cassandra/db/Memtable.java  | 10 ++--
 .../db/MigrationRequestVerbHandler.java         |  2 +-
 .../cassandra/db/SchemaCheckVerbHandler.java    |  2 +-
 .../cassandra/db/SizeEstimatesRecorder.java     |  6 +--
 .../cassandra/db/SliceFromReadCommand.java      |  2 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  2 +-
 .../cassandra/db/commitlog/CommitLog.java       |  6 +--
 .../db/commitlog/CommitLogArchiver.java         |  4 +-
 .../db/commitlog/CommitLogReplayer.java         | 22 ++++----
 .../db/commitlog/CommitLogSegmentManager.java   | 14 ++---
 .../db/compaction/CompactionController.java     |  4 +-
 .../db/compaction/CompactionManager.java        | 30 +++++------
 .../cassandra/db/compaction/CompactionTask.java |  8 +--
 .../DateTieredCompactionStrategy.java           | 10 ++--
 .../compaction/LeveledCompactionStrategy.java   |  4 +-
 .../db/compaction/LeveledManifest.java          | 32 +++++------
 .../SizeTieredCompactionStrategy.java           |  4 +-
 .../compaction/WrappingCompactionStrategy.java  |  2 +-
 .../writers/DefaultCompactionWriter.java        |  2 +-
 .../SplittingSizeTieredCompactionWriter.java    |  4 +-
 .../cassandra/db/filter/ExtendedFilter.java     |  2 +-
 .../AbstractSimplePerColumnSecondaryIndex.java  |  8 +--
 .../db/index/composites/CompositesIndex.java    |  4 +-
 .../db/index/composites/CompositesSearcher.java |  6 +--
 .../cassandra/db/index/keys/KeysSearcher.java   |  4 +-
 .../db/lifecycle/LifecycleTransaction.java      | 20 +++----
 .../apache/cassandra/db/lifecycle/Tracker.java  |  8 +--
 .../org/apache/cassandra/dht/BootStrapper.java  |  4 +-
 .../org/apache/cassandra/dht/RangeStreamer.java | 12 ++---
 .../hadoop/AbstractColumnFamilyInputFormat.java |  4 +-
 .../hadoop/ColumnFamilyInputFormat.java         |  4 +-
 .../hadoop/ColumnFamilyOutputFormat.java        |  4 +-
 .../hadoop/ColumnFamilyRecordReader.java        |  8 +--
 .../cassandra/hadoop/cql3/CqlRecordReader.java  |  6 +--
 ...mitedLocalNodeFirstLocalBalancingPolicy.java | 14 ++---
 .../cassandra/hadoop/pig/CassandraStorage.java  |  2 +-
 .../cassandra/hadoop/pig/CqlNativeStorage.java  |  2 +-
 .../io/sstable/IndexSummaryManager.java         | 10 ++--
 .../apache/cassandra/io/sstable/SSTable.java    |  2 +-
 .../io/sstable/format/SSTableReader.java        | 24 ++++-----
 .../io/sstable/metadata/MetadataSerializer.java |  8 +--
 .../org/apache/cassandra/io/util/FileUtils.java |  8 +--
 .../cassandra/io/util/MmappedSegmentedFile.java |  2 +-
 .../locator/AbstractReplicationStrategy.java    |  2 +-
 .../locator/NetworkTopologyStrategy.java        |  2 +-
 .../cassandra/locator/PropertyFileSnitch.java   |  6 +--
 .../locator/ReconnectableSnitchHelper.java      |  2 +-
 .../apache/cassandra/locator/TokenMetadata.java |  8 +--
 .../net/IncomingStreamingConnection.java        |  4 +-
 .../cassandra/net/IncomingTcpConnection.java    | 10 ++--
 .../cassandra/net/MessageDeliveryTask.java      |  2 +-
 .../apache/cassandra/net/MessagingService.java  | 20 +++----
 .../cassandra/net/OutboundTcpConnection.java    | 12 ++---
 .../cassandra/net/ResponseVerbHandler.java      |  2 +-
 .../cassandra/schema/LegacySchemaTables.java    |  2 +-
 .../cassandra/service/FileCacheService.java     |  4 +-
 .../apache/cassandra/service/GCInspector.java   |  4 +-
 .../cassandra/service/LoadBroadcaster.java      |  4 +-
 .../apache/cassandra/service/ReadCallback.java  | 12 ++---
 .../cassandra/service/RowDataResolver.java      | 12 ++---
 .../cassandra/service/RowDigestResolver.java    |  8 +--
 .../apache/cassandra/service/StorageProxy.java  | 14 ++---
 .../cassandra/thrift/CassandraServer.java       | 56 ++++++++++----------
 .../thrift/CustomTThreadPoolServer.java         |  4 +-
 .../cassandra/thrift/ThriftValidation.java      |  4 +-
 .../org/apache/cassandra/tracing/Tracing.java   |  2 +-
 .../org/apache/cassandra/transport/Message.java |  6 +--
 .../cassandra/triggers/CustomClassLoader.java   |  2 +-
 .../org/apache/cassandra/utils/CLibrary.java    |  2 +-
 .../cassandra/utils/EstimatedHistogram.java     |  4 +-
 .../org/apache/cassandra/utils/Mx4jTool.java    |  4 +-
 .../apache/cassandra/utils/OutputHandler.java   |  2 +-
 .../org/apache/cassandra/utils/TopKSampler.java |  2 +-
 97 files changed, 411 insertions(+), 360 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 78d90f6..6bd0a77 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -31,12 +31,22 @@ Changed Defaults
      providing the '-full' parameter to nodetool repair.
    - Parallel repairs are the default since 2.2.0, run sequential repairs
      by providing the '-seq' parameter to nodetool repair.
+   - The following INFO logs were reduced to DEBUG level and will now show
+     on debug.log instead of system.log:
+      - Memtable flushing actions
+      - Commit log replayed files
+      - Compacted sstables
+      - SStable opening (SSTableReader)
 
 New features
 ------------
    - Custom QueryHandlers can retrieve the column specifications for the bound
      variables from QueryOptions by using the hasColumnSpecifications()
      and getColumnSpecifications() methods.
+   - A new default assynchronous log appender debug.log was created in addition
+     to  the system.log appender in order to provide more detailed log debugging.
+     In order to disable debug logging, you must comment-out the ASYNCDEBUGLOG
+     appender on conf/logback.xml. See CASSANDRA-10241 for more information.
 
 
 2.2.1

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/conf/logback.xml
----------------------------------------------------------------------
diff --git a/conf/logback.xml b/conf/logback.xml
index eb2dcd1..a47740d 100644
--- a/conf/logback.xml
+++ b/conf/logback.xml
@@ -17,28 +17,67 @@
  under the License.
 -->
 
+<!--
+In order to disable debug.log, comment-out the ASYNCDEBUGLOG
+appender reference in the root level section below.
+-->
+
 <configuration scan="true">
   <jmxConfigurator />
-  <appender name="FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+  <shutdownHook class="ch.qos.logback.core.hook.DelayingShutdownHook"/>
+
+  <!-- SYSTEMLOG rolling file appender to system.log (INFO level) -->
+
+  <appender name="SYSTEMLOG" class="ch.qos.logback.core.rolling.RollingFileAppender">
+    <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
+      <level>INFO</level>
+    </filter>
     <file>${cassandra.logdir}/system.log</file>
     <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
       <fileNamePattern>${cassandra.logdir}/system.log.%i.zip</fileNamePattern>
       <minIndex>1</minIndex>
       <maxIndex>20</maxIndex>
     </rollingPolicy>
+    <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
+      <maxFileSize>20MB</maxFileSize>
+    </triggeringPolicy>
+    <encoder>
+      <pattern>%-5level [%thread] %date{ISO8601} %F:%L - %msg%n</pattern>
+    </encoder>
+  </appender>
 
+  <!-- DEBUGLOG rolling file appender to debug.log (all levels) -->
+
+  <appender name="DEBUGLOG" class="ch.qos.logback.core.rolling.RollingFileAppender">
+    <file>${cassandra.logdir}/debug.log</file>
+    <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
+      <fileNamePattern>${cassandra.logdir}/debug.log.%i.zip</fileNamePattern>
+      <minIndex>1</minIndex>
+      <maxIndex>20</maxIndex>
+    </rollingPolicy>
     <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
       <maxFileSize>20MB</maxFileSize>
     </triggeringPolicy>
     <encoder>
       <pattern>%-5level [%thread] %date{ISO8601} %F:%L - %msg%n</pattern>
-      <!-- old-style log format
-      <pattern>%5level [%thread] %date{ISO8601} %F (line %L) %msg%n</pattern>
-      -->
     </encoder>
   </appender>
-  
+
+  <!-- ASYNCLOG assynchronous appender to debug.log (all levels) -->
+
+  <appender name="ASYNCDEBUGLOG" class="ch.qos.logback.classic.AsyncAppender">
+    <queueSize>1024</queueSize>
+    <discardingThreshold>0</discardingThreshold>
+    <includeCallerData>true</includeCallerData>
+    <appender-ref ref="DEBUGLOG" />
+  </appender>
+
+  <!-- STDOUT console appender to stdout (INFO level) -->
+
   <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
+      <level>INFO</level>
+    </filter>
     <encoder>
       <pattern>%-5level %date{HH:mm:ss,SSS} %msg%n</pattern>
     </encoder>
@@ -49,12 +88,14 @@
    -->
 
   <root level="INFO">
-    <appender-ref ref="FILE" />
+    <appender-ref ref="SYSTEMLOG" />
     <appender-ref ref="STDOUT" />
+    <appender-ref ref="ASYNCDEBUGLOG" /> <!-- Comment this line to disable debug.log -->
     <!--
     <appender-ref ref="LogbackMetrics" />
     -->
   </root>
-  
+
+  <logger name="org.apache.cassandra" level="DEBUG"/>
   <logger name="com.thinkaurelius.thrift" level="ERROR"/>
 </configuration>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index 5cdea3f..88069a2 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -439,7 +439,7 @@ public class CassandraAuthorizer implements IAuthorizer
         {
             logger.info("Unable to complete conversion of legacy permissions data (perhaps not enough nodes are upgraded yet). " +
                         "Conversion should not be considered complete");
-            logger.debug("Conversion error", e);
+            logger.trace("Conversion error", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index 9151958..3a59581 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -376,7 +376,7 @@ public class CassandraRoleManager implements IRoleManager
                 // will be finished by then.
                 if (!MessagingService.instance().areAllNodesAtLeast22())
                 {
-                    logger.debug("Not all nodes are upgraded to a version that supports Roles yet, rescheduling setup task");
+                    logger.trace("Not all nodes are upgraded to a version that supports Roles yet, rescheduling setup task");
                     scheduleSetupTask(setupTask);
                     return;
                 }
@@ -442,7 +442,7 @@ public class CassandraRoleManager implements IRoleManager
         {
             logger.info("Unable to complete conversion of legacy auth data (perhaps not enough nodes are upgraded yet). " +
                         "Conversion should not be considered complete");
-            logger.debug("Conversion error", e);
+            logger.trace("Conversion error", e);
             throw e;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index 87bc073..c0d2283 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -86,7 +86,7 @@ public class PasswordAuthenticator implements IAuthenticator
         }
         catch (RequestExecutionException e)
         {
-            logger.debug("Error performing internal authentication", e);
+            logger.trace("Error performing internal authentication", e);
             throw new AuthenticationException(e.toString());
         }
     }
@@ -196,7 +196,7 @@ public class PasswordAuthenticator implements IAuthenticator
          */
         private void decodeCredentials(byte[] bytes) throws AuthenticationException
         {
-            logger.debug("Decoding credentials from client token");
+            logger.trace("Decoding credentials from client token");
             byte[] user = null;
             byte[] pass = null;
             int end = bytes.length;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/auth/PermissionsCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/PermissionsCache.java b/src/java/org/apache/cassandra/auth/PermissionsCache.java
index bc96d82..8746b36 100644
--- a/src/java/org/apache/cassandra/auth/PermissionsCache.java
+++ b/src/java/org/apache/cassandra/auth/PermissionsCache.java
@@ -137,7 +137,7 @@ public class PermissionsCache implements PermissionsCacheMBean
                                            }
                                            catch (Exception e)
                                            {
-                                               logger.debug("Error performing async refresh of user permissions", e);
+                                               logger.trace("Error performing async refresh of user permissions", e);
                                                throw e;
                                            }
                                        }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/auth/RolesCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/RolesCache.java b/src/java/org/apache/cassandra/auth/RolesCache.java
index 58aa739..554df9e 100644
--- a/src/java/org/apache/cassandra/auth/RolesCache.java
+++ b/src/java/org/apache/cassandra/auth/RolesCache.java
@@ -135,7 +135,7 @@ public class RolesCache implements RolesCacheMBean
                                     return roleManager.getRoles(primaryRole, true);
                                 } catch (Exception e)
                                 {
-                                    logger.debug("Error performing async refresh of user roles", e);
+                                    logger.trace("Error performing async refresh of user roles", e);
                                     throw e;
                                 }
                             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/cache/AutoSavingCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index 48d60b7..c08925d 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -256,8 +256,8 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
                 FileUtils.closeQuietly(in);
             }
         }
-        if (logger.isDebugEnabled())
-            logger.debug("completed reading ({} ms; {} keys) saved cache {}",
+        if (logger.isTraceEnabled())
+            logger.trace("completed reading ({} ms; {} keys) saved cache {}",
                     TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start), count, dataPath);
         return count;
     }
@@ -320,12 +320,12 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
 
         public void saveCache()
         {
-            logger.debug("Deleting old {} files.", cacheType);
+            logger.trace("Deleting old {} files.", cacheType);
             deleteOldCacheFiles();
 
             if (!keyIterator.hasNext())
             {
-                logger.debug("Skipping {} save, cache is empty.", cacheType);
+                logger.trace("Skipping {} save, cache is empty.", cacheType);
                 return;
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/cache/SerializingCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/SerializingCache.java b/src/java/org/apache/cassandra/cache/SerializingCache.java
index 0e38922..01d70b4 100644
--- a/src/java/org/apache/cassandra/cache/SerializingCache.java
+++ b/src/java/org/apache/cassandra/cache/SerializingCache.java
@@ -92,7 +92,7 @@ public class SerializingCache<K, V> implements ICache<K, V>
         }
         catch (IOException e)
         {
-            logger.debug("Cannot fetch in memory data, we will fallback to read from disk ", e);
+            logger.trace("Cannot fetch in memory data, we will fallback to read from disk ", e);
             return null;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/client/RingCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/client/RingCache.java b/src/java/org/apache/cassandra/client/RingCache.java
index 094cf4f..5196bce 100644
--- a/src/java/org/apache/cassandra/client/RingCache.java
+++ b/src/java/org/apache/cassandra/client/RingCache.java
@@ -93,7 +93,7 @@ public class RingCache
         }
         catch (TException e)
         {
-            logger.debug("Error contacting seed list {} {}", ConfigHelper.getOutputInitialAddress(conf), e.getMessage());
+            logger.trace("Error contacting seed list {} {}", ConfigHelper.getOutputInitialAddress(conf), e.getMessage());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java
index ea0715c..a722b87 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java
@@ -54,7 +54,7 @@ public class DebuggableScheduledThreadPoolExecutor extends ScheduledThreadPoolEx
                 if (task instanceof Future)
                     ((Future) task).cancel(false);
 
-                logger.debug("ScheduledThreadPoolExecutor has shut down as part of C* shutdown");
+                logger.trace("ScheduledThreadPoolExecutor has shut down as part of C* shutdown");
             }
             else
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
index fe6cade..a6d0049 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
@@ -266,7 +266,7 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
             }
             catch (CancellationException e)
             {
-                logger.debug("Task cancelled", e);
+                logger.trace("Task cancelled", e);
             }
             catch (ExecutionException e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 161d8d0..30a111d 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -618,20 +618,20 @@ public class QueryProcessor implements QueryHandler
 
         public void onUpdateColumnFamily(String ksName, String cfName, boolean columnsDidChange)
         {
-            logger.debug("Column definitions for {}.{} changed, invalidating related prepared statements", ksName, cfName);
+            logger.trace("Column definitions for {}.{} changed, invalidating related prepared statements", ksName, cfName);
             if (columnsDidChange)
                 removeInvalidPreparedStatements(ksName, cfName);
         }
 
         public void onDropKeyspace(String ksName)
         {
-            logger.debug("Keyspace {} was dropped, invalidating related prepared statements", ksName);
+            logger.trace("Keyspace {} was dropped, invalidating related prepared statements", ksName);
             removeInvalidPreparedStatements(ksName, null);
         }
 
         public void onDropColumnFamily(String ksName, String cfName)
         {
-            logger.debug("Table {}.{} was dropped, invalidating related prepared statements", ksName, cfName);
+            logger.trace("Table {}.{} was dropped, invalidating related prepared statements", ksName, cfName);
             removeInvalidPreparedStatements(ksName, cfName);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
index 97a08b1..c40e031 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
@@ -170,7 +170,7 @@ public final class JavaSourceUDFFactory
 
         String javaSource = javaSourceBuilder.toString();
 
-        logger.debug("Compiling Java source UDF '{}' as class '{}' using source:\n{}", name, targetClassName, javaSource);
+        logger.trace("Compiling Java source UDF '{}' as class '{}' using source:\n{}", name, targetClassName, javaSource);
 
         try
         {
@@ -303,7 +303,7 @@ public final class JavaSourceUDFFactory
             if (i > 0)
                 code.append(",\n");
 
-            if (logger.isDebugEnabled())
+            if (logger.isTraceEnabled())
                 code.append("                /* parameter '").append(argNames.get(i)).append("' */\n");
 
             code

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
index 4d9a79f..e55d450 100644
--- a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
+++ b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
@@ -143,7 +143,7 @@ public class ScriptBasedUDF extends UDFunction
         }
         catch (RuntimeException | ScriptException e)
         {
-            logger.debug("Execution of UDF '{}' failed", name, e);
+            logger.trace("Execution of UDF '{}' failed", name, e);
             throw FunctionExecutionException.create(this, e);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index c3b0993..edc092d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -164,7 +164,7 @@ public class CreateIndexStatement extends SchemaAlteringStatement
     {
         CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy();
         IndexTarget target = rawTarget.prepare(cfm);
-        logger.debug("Updating column {} definition for index {}", target.column, indexName);
+        logger.trace("Updating column {} definition for index {}", target.column, indexName);
         ColumnDefinition cd = cfm.getColumnDefinition(target.column);
 
         if (cd.getIndexType() != null && ifNotExists)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java
index 1c1b37b..ba56f0d 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -163,7 +163,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
     private void replayAllFailedBatches() throws ExecutionException, InterruptedException
     {
-        logger.debug("Started replayAllFailedBatches");
+        logger.trace("Started replayAllFailedBatches");
 
         // rate limit is in bytes per second. Uses Double.MAX_VALUE if disabled (set to 0 in cassandra.yaml).
         // max rate is scaled by the number of nodes in the cluster (same as for HHOM - see CASSANDRA-5272).
@@ -191,7 +191,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
         cleanup();
 
-        logger.debug("Finished replayAllFailedBatches");
+        logger.trace("Finished replayAllFailedBatches");
     }
 
     private void deleteBatch(UUID id)
@@ -274,7 +274,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
         public int replay(RateLimiter rateLimiter) throws IOException
         {
-            logger.debug("Replaying batch {}", id);
+            logger.trace("Replaying batch {}", id);
 
             List<Mutation> mutations = replayingMutations();
 
@@ -303,8 +303,8 @@ public class BatchlogManager implements BatchlogManagerMBean
                 }
                 catch (WriteTimeoutException|WriteFailureException e)
                 {
-                    logger.debug("Failed replaying a batched mutation to a node, will write a hint");
-                    logger.debug("Failure was : {}", e.getMessage());
+                    logger.trace("Failed replaying a batched mutation to a node, will write a hint");
+                    logger.trace("Failure was : {}", e.getMessage());
                     // writing hints for the rest to hints, starting from i
                     writeHintsForUndeliveredEndpoints(i);
                     return;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index a8a8910..4b418b4 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -217,7 +217,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         int period = metadata.getMemtableFlushPeriod();
         if (period > 0)
         {
-            logger.debug("scheduling flush in {} ms", period);
+            logger.trace("scheduling flush in {} ms", period);
             WrappedRunnable runnable = new WrappedRunnable()
             {
                 protected void runMayThrow() throws Exception
@@ -422,7 +422,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 throw new RuntimeException(e);
             }
-            logger.debug("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
+            logger.trace("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
             latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
             {
                 public void run()
@@ -570,7 +570,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             }
         }
 
-        logger.debug("Removing compacted SSTable files from {} (see http://wiki.apache.org/cassandra/MemtableSSTable)", metadata.cfName);
+        logger.trace("Removing compacted SSTable files from {} (see http://wiki.apache.org/cassandra/MemtableSSTable)", metadata.cfName);
 
         for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister().list().entrySet())
         {
@@ -649,7 +649,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         {
             HashSet<Integer> missingGenerations = new HashSet<>(unfinishedGenerations);
             missingGenerations.removeAll(allGenerations);
-            logger.debug("Unfinished compactions of {}.{} reference missing sstables of generations {}",
+            logger.trace("Unfinished compactions of {}.{} reference missing sstables of generations {}",
                          metadata.ksName, metadata.cfName, missingGenerations);
         }
 
@@ -682,7 +682,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 // any of the ancestors would work, so we'll just lookup the compaction task ID with the first one
                 UUID compactionTaskID = unfinishedCompactions.get(ancestors.iterator().next());
                 assert compactionTaskID != null;
-                logger.debug("Going to delete unfinished compaction product {}", desc);
+                logger.trace("Going to delete unfinished compaction product {}", desc);
                 SSTable.delete(desc, sstableFiles.getValue());
                 SystemKeyspace.finishCompaction(compactionTaskID);
             }
@@ -699,7 +699,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             if (completedAncestors.contains(desc.generation))
             {
                 // if any of the ancestors were participating in a compaction, finish that compaction
-                logger.debug("Going to delete leftover compaction ancestor {}", desc);
+                logger.trace("Going to delete leftover compaction ancestor {}", desc);
                 SSTable.delete(desc, sstableFiles.getValue());
                 UUID compactionTaskID = unfinishedCompactions.get(desc.generation);
                 if (compactionTaskID != null)
@@ -916,7 +916,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             }
         }
 
-        logger.info("Enqueuing flush of {}: {}", name, String.format("%d (%.0f%%) on-heap, %d (%.0f%%) off-heap",
+        logger.debug("Enqueuing flush of {}: {}", name, String.format("%d (%.0f%%) on-heap, %d (%.0f%%) off-heap",
                                                                      onHeapTotal, onHeapRatio * 100, offHeapTotal, offHeapRatio * 100));
     }
 
@@ -955,7 +955,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 {
                     public void run()
                     {
-                        logger.debug("forceFlush requested but everything is clean in {}", name);
+                        logger.trace("forceFlush requested but everything is clean in {}", name);
                     }
                 }, null);
                 postFlushExecutor.execute(task);
@@ -1208,7 +1208,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 float flushingOffHeap = Memtable.MEMORY_POOL.offHeap.reclaimingRatio();
                 float thisOnHeap = largest.getAllocator().onHeap().ownershipRatio();
                 float thisOffHeap = largest.getAllocator().onHeap().ownershipRatio();
-                logger.info("Flushing largest {} to free up room. Used total: {}, live: {}, flushing: {}, this: {}",
+                logger.debug("Flushing largest {} to free up room. Used total: {}, live: {}, flushing: {}, this: {}",
                             largest.cfs, ratio(usedOnHeap, usedOffHeap), ratio(liveOnHeap, liveOffHeap),
                             ratio(flushingOnHeap, flushingOffHeap), ratio(thisOnHeap, thisOffHeap));
                 largest.cfs.switchMemtableIfCurrent(largest);
@@ -1343,7 +1343,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public Collection<SSTableReader> getOverlappingSSTables(Iterable<SSTableReader> sstables)
     {
-        logger.debug("Checking for sstables overlapping {}", sstables);
+        logger.trace("Checking for sstables overlapping {}", sstables);
 
         // a normal compaction won't ever have an empty sstables list, but we create a skeleton
         // compaction controller for streaming, and that passes an empty list.
@@ -1972,7 +1972,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     }
                 }
 
-                logger.debug("ViewFilter for {}/{} sstables", sstables.size(), getSSTables().size());
+                logger.trace("ViewFilter for {}/{} sstables", sstables.size(), getSSTables().size());
                 return ImmutableList.copyOf(sstables);
             }
         };
@@ -2328,8 +2328,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     ssTable.createLinks(snapshotDirectory.getPath()); // hard links
                     filesJSONArr.add(ssTable.descriptor.relativeFilenameFor(Component.DATA));
 
-                    if (logger.isDebugEnabled())
-                        logger.debug("Snapshot for {} keyspace data file {} created in {}", keyspace, ssTable.getFilename(), snapshotDirectory);
+                    if (logger.isTraceEnabled())
+                        logger.trace("Snapshot for {} keyspace data file {} created in {}", keyspace, ssTable.getFilename(), snapshotDirectory);
                     snapshottedSSTables.add(ssTable);
                 }
 
@@ -2373,7 +2373,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 ephemeralSnapshotMarker.getParentFile().mkdirs();
 
             Files.createFile(ephemeralSnapshotMarker.toPath());
-            logger.debug("Created ephemeral snapshot marker file on {}.", ephemeralSnapshotMarker.getAbsolutePath());
+            logger.trace("Created ephemeral snapshot marker file on {}.", ephemeralSnapshotMarker.getAbsolutePath());
         }
         catch (IOException e)
         {
@@ -2388,7 +2388,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         for (String ephemeralSnapshot : directories.listEphemeralSnapshots())
         {
-            logger.debug("Clearing ephemeral snapshot {} leftover from previous session.", ephemeralSnapshot);
+            logger.trace("Clearing ephemeral snapshot {} leftover from previous session.", ephemeralSnapshot);
             Directories.clearSnapshot(ephemeralSnapshot, directories.getCFDirectories());
         }
     }
@@ -2409,17 +2409,17 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 SSTableReader sstable = active.get(entries.getKey().generation);
                 if (sstable == null || !refs.tryRef(sstable))
                 {
-                    if (logger.isDebugEnabled())
-                        logger.debug("using snapshot sstable {}", entries.getKey());
+                    if (logger.isTraceEnabled())
+                        logger.trace("using snapshot sstable {}", entries.getKey());
                     // open without tracking hotness
                     sstable = SSTableReader.open(entries.getKey(), entries.getValue(), metadata, partitioner, true, false);
                     refs.tryRef(sstable);
                     // release the self ref as we never add the snapshot sstable to DataTracker where it is otherwise released
                     sstable.selfRef().release();
                 }
-                else if (logger.isDebugEnabled())
+                else if (logger.isTraceEnabled())
                 {
-                    logger.debug("using active sstable {}", entries.getKey());
+                    logger.trace("using active sstable {}", entries.getKey());
                 }
             }
         }
@@ -2634,7 +2634,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         // beginning if we restart before they [the CL segments] are discarded for
         // normal reasons post-truncate.  To prevent this, we store truncation
         // position in the System keyspace.
-        logger.debug("truncating {}", name);
+        logger.trace("truncating {}", name);
 
         if (keyspace.getMetadata().durableWrites || DatabaseDescriptor.isAutoSnapshot())
         {
@@ -2660,7 +2660,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         {
             public void run()
             {
-                logger.debug("Discarding sstable data for truncated CF + indexes");
+                logger.trace("Discarding sstable data for truncated CF + indexes");
 
                 final long truncatedAt = System.currentTimeMillis();
                 data.notifyTruncated(truncatedAt);
@@ -2674,13 +2674,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     index.truncateBlocking(truncatedAt);
 
                 SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
-                logger.debug("cleaning out row cache");
+                logger.trace("cleaning out row cache");
                 invalidateCaches();
             }
         };
 
         runWithCompactionsDisabled(Executors.callable(truncateRunnable), true);
-        logger.debug("truncate complete");
+        logger.trace("truncate complete");
     }
 
     public <V> V runWithCompactionsDisabled(Callable<V> callable, boolean interruptValidation)
@@ -2689,7 +2689,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         // and so we only run one major compaction at a time
         synchronized (this)
         {
-            logger.debug("Cancelling in-progress compactions for {}", metadata.cfName);
+            logger.trace("Cancelling in-progress compactions for {}", metadata.cfName);
 
             Iterable<ColumnFamilyStore> selfWithIndexes = concatWithIndexes();
             for (ColumnFamilyStore cfs : selfWithIndexes)
@@ -2709,7 +2709,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                         return null;
                     }
                 }
-                logger.debug("Compactions successfully cancelled");
+                logger.trace("Compactions successfully cancelled");
 
                 // run our task
                 try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/ConsistencyLevel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
index d49d66a..85ec0f3 100644
--- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
+++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
@@ -260,7 +260,7 @@ public enum ConsistencyLevel
                 int localLive = countLocalEndpoints(liveEndpoints);
                 if (localLive < blockFor)
                 {
-                    if (logger.isDebugEnabled())
+                    if (logger.isTraceEnabled())
                     {
                         StringBuilder builder = new StringBuilder("Local replicas [");
                         for (InetAddress endpoint : liveEndpoints)
@@ -269,7 +269,7 @@ public enum ConsistencyLevel
                                 builder.append(endpoint).append(",");
                         }
                         builder.append("] are insufficient to satisfy LOCAL_QUORUM requirement of ").append(blockFor).append(" live nodes in '").append(DatabaseDescriptor.getLocalDataCenter()).append("'");
-                        logger.debug(builder.toString());
+                        logger.trace(builder.toString());
                     }
                     throw new UnavailableException(this, blockFor, localLive);
                 }
@@ -291,7 +291,7 @@ public enum ConsistencyLevel
                 int live = Iterables.size(liveEndpoints);
                 if (live < blockFor)
                 {
-                    logger.debug("Live nodes {} do not satisfy ConsistencyLevel ({} required)", Iterables.toString(liveEndpoints), blockFor);
+                    logger.trace("Live nodes {} do not satisfy ConsistencyLevel ({} required)", Iterables.toString(liveEndpoints), blockFor);
                     throw new UnavailableException(this, blockFor, live);
                 }
                 break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
index d9ee38a..4dd8ac3 100644
--- a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
@@ -35,7 +35,7 @@ public class CounterMutationVerbHandler implements IVerbHandler<CounterMutation>
     public void doVerb(final MessageIn<CounterMutation> message, final int id)
     {
         final CounterMutation cm = message.payload;
-        logger.debug("Applying forwarded {}", cm);
+        logger.trace("Applying forwarded {}", cm);
 
         String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
         // We should not wait for the result of the write in this thread,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
index d5ede03..51d15b4 100644
--- a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
@@ -41,7 +41,7 @@ public class DefinitionsUpdateVerbHandler implements IVerbHandler<Collection<Mut
 
     public void doVerb(final MessageIn<Collection<Mutation>> message, int id)
     {
-        logger.debug("Received schema mutation push from {}", message.from);
+        logger.trace("Received schema mutation push from {}", message.from);
 
         StageManager.getStage(Stage.MIGRATION).submit(new WrappedRunnable()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 0011baf..066b759 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -258,7 +258,7 @@ public class Directories
                 for (File indexFile : indexFiles)
                 {
                     File destFile = new File(dataPath, indexFile.getName());
-                    logger.debug("Moving index file {} to {}", indexFile, destFile);
+                    logger.trace("Moving index file {} to {}", indexFile, destFile);
                     FileUtils.renameWithConfirm(indexFile, destFile);
                 }
             }
@@ -329,14 +329,14 @@ public class Directories
         {
             if (BlacklistedDirectories.isUnwritable(getLocationForDisk(dataDir)))
             {
-                logger.debug("removing blacklisted candidate {}", dataDir.location);
+                logger.trace("removing blacklisted candidate {}", dataDir.location);
                 continue;
             }
             DataDirectoryCandidate candidate = new DataDirectoryCandidate(dataDir);
             // exclude directory if its total writeSize does not fit to data directory
             if (candidate.availableSpace < writeSize)
             {
-                logger.debug("removing candidate {}, usable={}, requested={}", candidate.dataDirectory.location, candidate.availableSpace, writeSize);
+                logger.trace("removing candidate {}, usable={}, requested={}", candidate.dataDirectory.location, candidate.availableSpace, writeSize);
                 tooBig = true;
                 continue;
             }
@@ -728,7 +728,7 @@ public class Directories
             File snapshotDir = new File(dir, join(SNAPSHOT_SUBDIR, tag));
             if (snapshotDir.exists())
             {
-                logger.debug("Removing snapshot directory {}", snapshotDir);
+                logger.trace("Removing snapshot directory {}", snapshotDir);
                 try
                 {
                     FileUtils.deleteRecursive(snapshotDir);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index dae85b7..179c611 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -175,7 +175,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         {
             throw new RuntimeException(e);
         }
-        logger.debug("Created HHOM instance, registered MBean.");
+        logger.trace("Created HHOM instance, registered MBean.");
 
         Runnable runnable = new Runnable()
         {
@@ -317,7 +317,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         }
         if (gossiper.getEndpointStateForEndpoint(endpoint) == null)
             throw new TimeoutException("Node " + endpoint + " vanished while waiting for agreement");
-        logger.debug("schema for {} matches local schema", endpoint);
+        logger.trace("schema for {} matches local schema", endpoint);
         return waited;
     }
 
@@ -329,11 +329,11 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         // check if hints delivery has been paused
         if (hintedHandOffPaused)
         {
-            logger.debug("Hints delivery process is paused, aborting");
+            logger.trace("Hints delivery process is paused, aborting");
             return;
         }
 
-        logger.debug("Checking remote({}) schema before delivering hints", endpoint);
+        logger.trace("Checking remote({}) schema before delivering hints", endpoint);
         try
         {
             waitForSchemaAgreement(endpoint);
@@ -345,7 +345,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
 
         if (!FailureDetector.instance.isAlive(endpoint))
         {
-            logger.debug("Endpoint {} died before hint delivery, aborting", endpoint);
+            logger.trace("Endpoint {} died before hint delivery, aborting", endpoint);
             return;
         }
 
@@ -370,7 +370,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         Composite startColumn = Composites.EMPTY;
 
         int pageSize = calculatePageSize();
-        logger.debug("Using pageSize of {}", pageSize);
+        logger.trace("Using pageSize of {}", pageSize);
 
         // rate limit is in bytes per second. Uses Double.MAX_VALUE if disabled (set to 0 in cassandra.yaml).
         // max rate is scaled by the number of nodes in the cluster (CASSANDRA-5272).
@@ -411,7 +411,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                 // check if hints delivery has been paused during the process
                 if (hintedHandOffPaused)
                 {
-                    logger.debug("Hints delivery process is paused, aborting");
+                    logger.trace("Hints delivery process is paused, aborting");
                     break delivery;
                 }
 
@@ -434,7 +434,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                 }
                 catch (UnknownColumnFamilyException e)
                 {
-                    logger.debug("Skipping delivery of hint for deleted table", e);
+                    logger.trace("Skipping delivery of hint for deleted table", e);
                     deleteHint(hostIdBytes, hint.name(), hint.timestamp());
                     continue;
                 }
@@ -447,7 +447,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                 {
                     if (hint.timestamp() <= SystemKeyspace.getTruncatedAt(cfId))
                     {
-                        logger.debug("Skipping delivery of hint for truncated table {}", cfId);
+                        logger.trace("Skipping delivery of hint for truncated table {}", cfId);
                         mutation = mutation.without(cfId);
                     }
                 }
@@ -513,7 +513,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
      */
     private void scheduleAllDeliveries()
     {
-        logger.debug("Started scheduleAllDeliveries");
+        logger.trace("Started scheduleAllDeliveries");
 
         // Force a major compaction to get rid of the tombstones and expired hints. Do it once, before we schedule any
         // individual replay, to avoid N - 1 redundant individual compactions (when N is the number of nodes with hints
@@ -534,7 +534,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                 scheduleHintDelivery(target, false);
         }
 
-        logger.debug("Finished scheduleAllDeliveries");
+        logger.trace("Finished scheduleAllDeliveries");
     }
 
     /*
@@ -548,7 +548,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         if (!queuedDeliveries.add(to))
             return;
 
-        logger.debug("Scheduling delivery of Hints to {}", to);
+        logger.trace("Scheduling delivery of Hints to {}", to);
 
         hintDeliveryExecutor.execute(new Runnable()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index feb3c5e..92a0950 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -266,7 +266,7 @@ public class Keyspace
         this.metric = new KeyspaceMetrics(this);
         for (CFMetaData cfm : new ArrayList<>(metadata.cfMetaData().values()))
         {
-            logger.debug("Initializing {}.{}", getName(), cfm.cfName);
+            logger.trace("Initializing {}.{}", getName(), cfm.cfName);
             initCf(cfm.cfId, cfm.cfName, loadSSTables);
         }
     }
@@ -420,8 +420,8 @@ public class Keyspace
      */
     public static void indexRow(DecoratedKey key, ColumnFamilyStore cfs, Set<String> idxNames)
     {
-        if (logger.isDebugEnabled())
-            logger.debug("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.getKey()));
+        if (logger.isTraceEnabled())
+            logger.trace("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.getKey()));
 
         try (OpOrder.Group opGroup = cfs.keyspace.writeOrder.start())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index 1ce2b9f..e96a71e 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -360,13 +360,13 @@ public class Memtable implements Comparable<Memtable>
 
         private SSTableReader writeSortedContents(ReplayPosition context, File sstableDirectory)
         {
-            logger.info("Writing {}", Memtable.this.toString());
+            logger.debug("Writing {}", Memtable.this.toString());
 
             SSTableReader ssTable;
             // errors when creating the writer that may leave empty temp files.
             try (SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory)))
             {
-                boolean trackContention = logger.isDebugEnabled();
+                boolean trackContention = logger.isTraceEnabled();
                 int heavilyContendedRowCount = 0;
                 // (we can't clear out the map as-we-go to free up memory,
                 //  since the memtable is being used for queries in the "pending flush" category)
@@ -394,7 +394,7 @@ public class Memtable implements Comparable<Memtable>
 
                 if (writer.getFilePointer() > 0)
                 {
-                    logger.info(String.format("Completed flushing %s (%s) for commitlog position %s",
+                    logger.debug(String.format("Completed flushing %s (%s) for commitlog position %s",
                                               writer.getFilename(),
                                               FBUtilities.prettyPrintMemory(writer.getOnDiskFilePointer()),
                                               context));
@@ -404,14 +404,14 @@ public class Memtable implements Comparable<Memtable>
                 }
                 else
                 {
-                    logger.info("Completed flushing {}; nothing needed to be retained.  Commitlog position was {}",
+                    logger.debug("Completed flushing {}; nothing needed to be retained.  Commitlog position was {}",
                                 writer.getFilename(), context);
                     writer.abort();
                     ssTable = null;
                 }
 
                 if (heavilyContendedRowCount > 0)
-                    logger.debug(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, rows.size(), Memtable.this.toString()));
+                    logger.trace(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, rows.size(), Memtable.this.toString()));
 
                 return ssTable;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
index 79753c1..ab934c6 100644
--- a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
@@ -39,7 +39,7 @@ public class MigrationRequestVerbHandler implements IVerbHandler
 
     public void doVerb(MessageIn message, int id)
     {
-        logger.debug("Received migration request from {}.", message.from);
+        logger.trace("Received migration request from {}.", message.from);
         MessageOut<Collection<Mutation>> response = new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE,
                                                                      LegacySchemaTables.convertSchemaToMutations(),
                                                                      MigrationManager.MigrationsSerializer.instance);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java b/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
index 1a1f7a9..4270a24 100644
--- a/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
@@ -35,7 +35,7 @@ public class SchemaCheckVerbHandler implements IVerbHandler
 
     public void doVerb(MessageIn message, int id)
     {
-        logger.debug("Received schema check request.");
+        logger.trace("Received schema check request.");
         MessageOut<UUID> response = new MessageOut<UUID>(MessagingService.Verb.INTERNAL_RESPONSE, Schema.instance.getVersion(), UUIDSerializer.serializer);
         MessagingService.instance().sendReply(response, id, message.from);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
index f054315..8bf1ef3 100644
--- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
+++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
@@ -57,11 +57,11 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable
     {
         if (StorageService.instance.isStarting())
         {
-            logger.debug("Node has not yet joined; not recording size estimates");
+            logger.trace("Node has not yet joined; not recording size estimates");
             return;
         }
 
-        logger.debug("Recording size estimates");
+        logger.trace("Recording size estimates");
 
         // find primary token ranges for the local node.
         Collection<Token> localTokens = StorageService.instance.getLocalTokens();
@@ -74,7 +74,7 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable
                 long start = System.nanoTime();
                 recordSizeEstimates(table, localRanges);
                 long passed = System.nanoTime() - start;
-                logger.debug("Spent {} milliseconds on estimating {}.{} size",
+                logger.trace("Spent {} milliseconds on estimating {}.{} size",
                              TimeUnit.NANOSECONDS.toMillis(passed),
                              table.metadata.ksName,
                              table.metadata.cfName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SliceFromReadCommand.java b/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
index 461a3a1..edace9d 100644
--- a/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
@@ -65,7 +65,7 @@ public class SliceFromReadCommand extends ReadCommand
         // reads in order to guarantee that the static columns are fetched.  See CASSANDRA-8502 for more details.
         if (filter.reversed && filter.hasStaticSlice(cfm))
         {
-            logger.debug("Splitting reversed slice with static columns into two reads");
+            logger.trace("Splitting reversed slice with static columns into two reads");
             Pair<SliceQueryFilter, SliceQueryFilter> newFilters = filter.splitOutStaticSlice(cfm);
 
             Row normalResults =  keyspace.getRow(new QueryFilter(dk, cfName, newFilters.right, timestamp));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 18af90f..308edcd 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -1110,7 +1110,7 @@ public final class SystemKeyspace
             {
                 if (dataDirectory.getName().equals("Versions") && dataDirectory.listFiles().length > 0)
                 {
-                    logger.debug("Found unreadable versions info in pre 1.2 system.Versions table");
+                    logger.trace("Found unreadable versions info in pre 1.2 system.Versions table");
                     return UNREADABLE_VERSION.toString();
                 }
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index 0e08efe..a60c135 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -303,7 +303,7 @@ public class CommitLog implements CommitLogMBean
      */
     public void discardCompletedSegments(final UUID cfId, final ReplayPosition context)
     {
-        logger.debug("discard completed log segments for {}, table {}", context, cfId);
+        logger.trace("discard completed log segments for {}, table {}", context, cfId);
 
         // Go thru the active segment files, which are ordered oldest to newest, marking the
         // flushed CF as clean, until we reach the segment file containing the ReplayPosition passed
@@ -316,12 +316,12 @@ public class CommitLog implements CommitLogMBean
 
             if (segment.isUnused())
             {
-                logger.debug("Commit log segment {} is unused", segment);
+                logger.trace("Commit log segment {} is unused", segment);
                 allocator.recycleSegment(segment);
             }
             else
             {
-                logger.debug("Not safe to delete{} commit log segment {}; dirty is {}",
+                logger.trace("Not safe to delete{} commit log segment {}; dirty is {}",
                         (iter.hasNext() ? "" : " active"), segment, segment.dirtyString());
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
index 4c615e0..b734573 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
@@ -83,7 +83,7 @@ public class CommitLogArchiver
         {
             if (stream == null)
             {
-                logger.debug("No commitlog_archiving properties found; archive + pitr will be disabled");
+                logger.trace("No commitlog_archiving properties found; archive + pitr will be disabled");
                 return disabled();
             }
             else
@@ -237,7 +237,7 @@ public class CommitLogArchiver
                 File toFile = new File(DatabaseDescriptor.getCommitLogLocation(), descriptor.fileName());
                 if (toFile.exists())
                 {
-                    logger.debug("Skipping restore of archive {} as the segment already exists in the restore location {}",
+                    logger.trace("Skipping restore of archive {} as the segment already exists in the restore location {}",
                                  fromFile.getPath(), toFile.getPath());
                     continue;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index 389b111..cb02a8c 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -136,7 +136,7 @@ public class CommitLogReplayer
             cfPositions.put(cfs.metadata.cfId, rp);
         }
         ReplayPosition globalPosition = replayPositionOrdering.min(cfPositions.values());
-        logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPositions));
+        logger.trace("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPositions));
         return new CommitLogReplayer(commitLog, globalPosition, cfPositions, replayFilter);
     }
 
@@ -154,7 +154,7 @@ public class CommitLogReplayer
 
         // wait for all the writes to finish on the mutation stage
         FBUtilities.waitOnFutures(futures);
-        logger.debug("Finished waiting on mutations from recovery");
+        logger.trace("Finished waiting on mutations from recovery");
 
         // flush replayed keyspaces
         futures.clear();
@@ -333,7 +333,7 @@ public class CommitLogReplayer
             {
                 int replayPos = replayEnd + CommitLogSegment.SYNC_MARKER_SIZE;
 
-                if (logger.isDebugEnabled())
+                if (logger.isTraceEnabled())
                     logger.trace("Replaying {} between {} and {}", file, reader.getFilePointer(), end);
                 if (compressor != null)
                 {
@@ -361,7 +361,7 @@ public class CommitLogReplayer
                     try
                     {
                         int compressedLength = end - start;
-                        if (logger.isDebugEnabled())
+                        if (logger.isTraceEnabled())
                             logger.trace("Decompressing {} between replay positions {} and {}",
                                          file,
                                          replayPos,
@@ -392,13 +392,13 @@ public class CommitLogReplayer
         finally
         {
             FileUtils.closeQuietly(reader);
-            logger.info("Finished reading {}", file);
+            logger.debug("Finished reading {}", file);
         }
     }
 
     public boolean logAndCheckIfShouldSkip(File file, CommitLogDescriptor desc)
     {
-        logger.info("Replaying {} (CL version {}, messaging version {}, compression {})",
+        logger.debug("Replaying {} (CL version {}, messaging version {}, compression {})",
                     file.getPath(),
                     desc.version,
                     desc.getMessagingVersion(),
@@ -406,7 +406,7 @@ public class CommitLogReplayer
 
         if (globalPosition.segment > desc.id)
         {
-            logger.debug("skipping replay of fully-flushed {}", file);
+            logger.trace("skipping replay of fully-flushed {}", file);
             return true;
         }
         return false;
@@ -423,7 +423,7 @@ public class CommitLogReplayer
         while (reader.getFilePointer() < end && !reader.isEOF())
         {
             long mutationStart = reader.getFilePointer();
-            if (logger.isDebugEnabled())
+            if (logger.isTraceEnabled())
                 logger.trace("Reading mutation at {}", mutationStart);
 
             long claimedCRC32;
@@ -434,7 +434,7 @@ public class CommitLogReplayer
                 serializedSize = reader.readInt();
                 if (serializedSize == LEGACY_END_OF_SEGMENT_MARKER)
                 {
-                    logger.debug("Encountered end of segment marker at {}", reader.getFilePointer());
+                    logger.trace("Encountered end of segment marker at {}", reader.getFilePointer());
                     return false;
                 }
 
@@ -551,8 +551,8 @@ public class CommitLogReplayer
             return;
         }
 
-        if (logger.isDebugEnabled())
-            logger.debug("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), ByteBufferUtil.bytesToHex(mutation.key()), "{" + StringUtils.join(mutation.getColumnFamilies().iterator(), ", ") + "}");
+        if (logger.isTraceEnabled())
+            logger.trace("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), ByteBufferUtil.bytesToHex(mutation.key()), "{" + StringUtils.join(mutation.getColumnFamilies().iterator(), ", ") + "}");
 
         Runnable runnable = new WrappedRunnable()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index 5918474..f71bb1b 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -117,7 +117,7 @@ public class CommitLogSegmentManager
                             // if we have no more work to do, check if we should create a new segment
                             if (availableSegments.isEmpty() && (activeSegments.isEmpty() || createReserveSegments))
                             {
-                                logger.debug("No segments in reserve; creating a fresh one");
+                                logger.trace("No segments in reserve; creating a fresh one");
                                 // TODO : some error handling in case we fail to create a new segment
                                 availableSegments.add(CommitLogSegment.createSegment(commitLog));
                                 hasAvailableSegments.signalAll();
@@ -354,7 +354,7 @@ public class CommitLogSegmentManager
     void recycleSegment(final File file)
     {
         // (don't decrease managed size, since this was never a "live" segment)
-        logger.debug("(Unopened) segment {} is no longer needed and will be deleted now", file);
+        logger.trace("(Unopened) segment {} is no longer needed and will be deleted now", file);
         FileUtils.deleteWithConfirm(file);
     }
 
@@ -365,7 +365,7 @@ public class CommitLogSegmentManager
      */
     private void discardSegment(final CommitLogSegment segment, final boolean deleteFile)
     {
-        logger.debug("Segment {} is no longer active and will be deleted {}", segment, deleteFile ? "now" : "by the archive script");
+        logger.trace("Segment {} is no longer active and will be deleted {}", segment, deleteFile ? "now" : "by the archive script");
 
         segmentManagementTasks.add(new Runnable()
         {
@@ -397,7 +397,7 @@ public class CommitLogSegmentManager
     {
         long total = DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 1024;
         long currentSize = size.get();
-        logger.debug("Total active commitlog segment space used is {} out of {}", currentSize, total);
+        logger.trace("Total active commitlog segment space used is {} out of {}", currentSize, total);
         return total - currentSize;
     }
 
@@ -446,7 +446,7 @@ public class CommitLogSegmentManager
                 {
                     // even though we remove the schema entry before a final flush when dropping a CF,
                     // it's still possible for a writer to race and finish his append after the flush.
-                    logger.debug("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
+                    logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
                     segment.markClean(dirtyCFId, segment.getContext());
                 }
                 else if (!flushes.containsKey(dirtyCFId))
@@ -469,7 +469,7 @@ public class CommitLogSegmentManager
      */
     public void stopUnsafe(boolean deleteSegments)
     {
-        logger.debug("CLSM closing and clearing existing commit log segments...");
+        logger.trace("CLSM closing and clearing existing commit log segments...");
         createReserveSegments = false;
 
         awaitManagementTasksCompletion();
@@ -498,7 +498,7 @@ public class CommitLogSegmentManager
 
         size.set(0L);
 
-        logger.debug("CLSM done with closing and clearing existing commit log segments.");
+        logger.trace("CLSM done with closing and clearing existing commit log segments.");
     }
 
     // Used by tests only.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index 81d8b7c..5f0a198 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -110,7 +110,7 @@ public class CompactionController implements AutoCloseable
      */
     public static Set<SSTableReader> getFullyExpiredSSTables(ColumnFamilyStore cfStore, Iterable<SSTableReader> compacting, Iterable<SSTableReader> overlapping, int gcBefore)
     {
-        logger.debug("Checking droppable sstables in {}", cfStore);
+        logger.trace("Checking droppable sstables in {}", cfStore);
 
         if (compacting == null)
             return Collections.<SSTableReader>emptySet();
@@ -150,7 +150,7 @@ public class CompactionController implements AutoCloseable
             }
             else
             {
-               logger.debug("Dropping expired SSTable {} (maxLocalDeletionTime={}, gcBefore={})",
+               logger.trace("Dropping expired SSTable {} (maxLocalDeletionTime={}, gcBefore={})",
                         candidate, candidate.getSSTableMetadata().maxLocalDeletionTime, gcBefore);
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 7def98d..ea20a1f 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -153,19 +153,19 @@ public class CompactionManager implements CompactionManagerMBean
     {
         if (cfs.isAutoCompactionDisabled())
         {
-            logger.debug("Autocompaction is disabled");
+            logger.trace("Autocompaction is disabled");
             return Collections.emptyList();
         }
 
         int count = compactingCF.count(cfs);
         if (count > 0 && executor.getActiveCount() >= executor.getMaximumPoolSize())
         {
-            logger.debug("Background compaction is still running for {}.{} ({} remaining). Skipping",
+            logger.trace("Background compaction is still running for {}.{} ({} remaining). Skipping",
                          cfs.keyspace.getName(), cfs.name, count);
             return Collections.emptyList();
         }
 
-        logger.debug("Scheduling a background task check for {}.{} with {}",
+        logger.trace("Scheduling a background task check for {}.{} with {}",
                      cfs.keyspace.getName(),
                      cfs.name,
                      cfs.getCompactionStrategy().getName());
@@ -211,10 +211,10 @@ public class CompactionManager implements CompactionManagerMBean
         {
             try
             {
-                logger.debug("Checking {}.{}", cfs.keyspace.getName(), cfs.name);
+                logger.trace("Checking {}.{}", cfs.keyspace.getName(), cfs.name);
                 if (!cfs.isValid())
                 {
-                    logger.debug("Aborting compaction for dropped CF");
+                    logger.trace("Aborting compaction for dropped CF");
                     return;
                 }
 
@@ -222,7 +222,7 @@ public class CompactionManager implements CompactionManagerMBean
                 AbstractCompactionTask task = strategy.getNextBackgroundTask(getDefaultGcBefore(cfs));
                 if (task == null)
                 {
-                    logger.debug("No tasks available");
+                    logger.trace("No tasks available");
                     return;
                 }
                 task.execute(metrics);
@@ -461,7 +461,7 @@ public class CompactionManager implements CompactionManagerMBean
                                       long repairedAt) throws InterruptedException, IOException
     {
         logger.info("Starting anticompaction for {}.{} on {}/{} sstables", cfs.keyspace.getName(), cfs.getColumnFamilyName(), validatedForRepair.size(), cfs.getSSTables().size());
-        logger.debug("Starting anticompaction for ranges {}", ranges);
+        logger.trace("Starting anticompaction for ranges {}", ranges);
         Set<SSTableReader> sstables = new HashSet<>(validatedForRepair);
         Set<SSTableReader> mutatedRepairStatuses = new HashSet<>();
         Set<SSTableReader> nonAnticompacting = new HashSet<>();
@@ -780,7 +780,7 @@ public class CompactionManager implements CompactionManagerMBean
         }
         if (!needsCleanup(sstable, ranges))
         {
-            logger.debug("Skipping {} for cleanup; all rows should be kept", sstable);
+            logger.trace("Skipping {} for cleanup; all rows should be kept", sstable);
             return;
         }
 
@@ -790,8 +790,8 @@ public class CompactionManager implements CompactionManagerMBean
 
         long expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(),
                                                SSTableReader.getApproximateKeyCount(txn.originals()));
-        if (logger.isDebugEnabled())
-            logger.debug("Expected bloom filter size : {}", expectedBloomFilterSize);
+        if (logger.isTraceEnabled())
+            logger.trace("Expected bloom filter size : {}", expectedBloomFilterSize);
 
         logger.info("Cleaning up {}", sstable);
 
@@ -1110,11 +1110,11 @@ public class CompactionManager implements CompactionManagerMBean
                 }
             }
 
-            if (logger.isDebugEnabled())
+            if (logger.isTraceEnabled())
             {
                 // MT serialize may take time
                 long duration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
-                logger.debug("Validation finished in {} msec, depth {} for {} keys, serialized size {} bytes for {}",
+                logger.trace("Validation finished in {} msec, depth {} for {} keys, serialized size {} bytes for {}",
                              duration,
                              depth,
                              numPartitions,
@@ -1243,7 +1243,7 @@ public class CompactionManager implements CompactionManagerMBean
             repairedSSTableWriter.commit();
             unRepairedSSTableWriter.commit();
 
-            logger.debug("Repaired {} keys out of {} for {}/{} in {}", repairedKeyCount,
+            logger.trace("Repaired {} keys out of {} for {}/{} in {}", repairedKeyCount,
                                                                        repairedKeyCount + unrepairedKeyCount,
                                                                        cfs.keyspace.getName(),
                                                                        cfs.getColumnFamilyName(),
@@ -1295,7 +1295,7 @@ public class CompactionManager implements CompactionManagerMBean
             {
                 if (!AutoSavingCache.flushInProgress.add(writer.cacheType()))
                 {
-                    logger.debug("Cache flushing was already in progress: skipping {}", writer.getCompactionInfo());
+                    logger.trace("Cache flushing was already in progress: skipping {}", writer.getCompactionInfo());
                     return;
                 }
                 try
@@ -1417,7 +1417,7 @@ public class CompactionManager implements CompactionManagerMBean
                     if (t.getSuppressed() != null && t.getSuppressed().length > 0)
                         logger.warn("Interruption of compaction encountered exceptions:", t);
                     else
-                        logger.debug("Full interruption stack trace:", t);
+                        logger.trace("Full interruption stack trace:", t);
                 }
                 else
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index ea64fb2..575c326 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -139,7 +139,7 @@ public class CompactionTask extends AbstractCompactionTask
         }
         ssTableLoggerMsg.append("]");
         String taskIdLoggerMsg = taskId == null ? UUIDGen.getTimeUUID().toString() : taskId.toString();
-        logger.info("Compacting ({}) {}", taskIdLoggerMsg, ssTableLoggerMsg);
+        logger.debug("Compacting ({}) {}", taskIdLoggerMsg, ssTableLoggerMsg);
 
         long start = System.nanoTime();
 
@@ -221,10 +221,10 @@ public class CompactionTask extends AbstractCompactionTask
             double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
             long totalSourceRows = 0;
             String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
-            logger.info(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
+            logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
                                       taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
-            logger.debug(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-            logger.debug("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
+            logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
+            logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
 
             if (offline)
                 Refs.release(Refs.selfRefs(newSStables));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a849efe/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
index 0956962..da2d35d 100644
--- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
@@ -50,10 +50,10 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         if (!options.containsKey(AbstractCompactionStrategy.TOMBSTONE_COMPACTION_INTERVAL_OPTION) && !options.containsKey(AbstractCompactionStrategy.TOMBSTONE_THRESHOLD_OPTION))
         {
             disableTombstoneCompactions = true;
-            logger.debug("Disabling tombstone compactions for DTCS");
+            logger.trace("Disabling tombstone compactions for DTCS");
         }
         else
-            logger.debug("Enabling tombstone compactions for DTCS");
+            logger.trace("Enabling tombstone compactions for DTCS");
 
     }
 
@@ -99,7 +99,7 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         List<SSTableReader> compactionCandidates = new ArrayList<>(getNextNonExpiredSSTables(Sets.difference(candidates, expired), gcBefore));
         if (!expired.isEmpty())
         {
-            logger.debug("Including expired sstables: {}", expired);
+            logger.trace("Including expired sstables: {}", expired);
             compactionCandidates.addAll(expired);
         }
         return compactionCandidates;
@@ -134,7 +134,7 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         Iterable<SSTableReader> candidates = filterOldSSTables(Lists.newArrayList(candidateSSTables), options.maxSSTableAge, now);
 
         List<List<SSTableReader>> buckets = getBuckets(createSSTableAndMinTimestampPairs(candidates), options.baseTime, base, now);
-        logger.debug("Compaction buckets are {}", buckets);
+        logger.trace("Compaction buckets are {}", buckets);
         updateEstimatedCompactionsByTasks(buckets);
         List<SSTableReader> mostInteresting = newestBucket(buckets,
                                                            cfs.getMinimumCompactionThreshold(),
@@ -391,7 +391,7 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         LifecycleTransaction modifier = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
         if (modifier == null)
         {
-            logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
+            logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
             return null;
         }
 


[9/9] cassandra git commit: Merge branch cassandra-3.0 into trunk

Posted by bl...@apache.org.
Merge branch cassandra-3.0 into trunk


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

Branch: refs/heads/trunk
Commit: a2b1b8abcbe58c86a0a16c6dbf5955d219955e59
Parents: 56d81ac aa60cde
Author: blerer <be...@datastax.com>
Authored: Tue Sep 22 22:11:50 2015 +0200
Committer: blerer <be...@datastax.com>
Committed: Tue Sep 22 22:12:11 2015 +0200

----------------------------------------------------------------------
 NEWS.txt                                        | 16 +++---
 conf/logback.xml                                | 55 ++++++++++++++++---
 .../cassandra/auth/CassandraAuthorizer.java     |  2 +-
 .../cassandra/auth/CassandraRoleManager.java    |  4 +-
 .../cassandra/auth/PasswordAuthenticator.java   |  4 +-
 .../apache/cassandra/auth/PermissionsCache.java |  2 +-
 .../org/apache/cassandra/auth/RolesCache.java   |  2 +-
 .../cassandra/batchlog/BatchlogManager.java     | 10 ++--
 .../batchlog/LegacyBatchlogMigrator.java        |  8 +--
 .../apache/cassandra/cache/AutoSavingCache.java |  8 +--
 .../cassandra/cache/SerializingCache.java       |  2 +-
 .../org/apache/cassandra/client/RingCache.java  |  2 +-
 .../DebuggableScheduledThreadPoolExecutor.java  |  2 +-
 .../DebuggableThreadPoolExecutor.java           |  2 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |  6 +--
 .../cql3/functions/JavaBasedUDFunction.java     |  4 +-
 .../cassandra/cql3/functions/UDFunction.java    |  2 +-
 .../cql3/statements/CreateIndexStatement.java   |  2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  | 44 +++++++--------
 .../apache/cassandra/db/ConsistencyLevel.java   |  6 +--
 .../db/CounterMutationVerbHandler.java          |  2 +-
 .../db/DefinitionsUpdateVerbHandler.java        |  2 +-
 .../org/apache/cassandra/db/Directories.java    |  8 +--
 src/java/org/apache/cassandra/db/Keyspace.java  |  8 +--
 src/java/org/apache/cassandra/db/Memtable.java  | 16 +++---
 .../db/MigrationRequestVerbHandler.java         |  2 +-
 .../cassandra/db/SchemaCheckVerbHandler.java    |  2 +-
 .../cassandra/db/SizeEstimatesRecorder.java     |  6 +--
 .../org/apache/cassandra/db/SystemKeyspace.java |  4 +-
 .../cassandra/db/commitlog/CommitLog.java       |  6 +--
 .../db/commitlog/CommitLogArchiver.java         |  4 +-
 .../db/commitlog/CommitLogReplayer.java         | 22 ++++----
 .../db/commitlog/CommitLogSegmentManager.java   | 14 ++---
 .../db/compaction/CompactionController.java     |  4 +-
 .../db/compaction/CompactionManager.java        | 30 +++++------
 .../compaction/CompactionStrategyManager.java   |  2 +-
 .../cassandra/db/compaction/CompactionTask.java |  8 +--
 .../DateTieredCompactionStrategy.java           | 10 ++--
 .../compaction/LeveledCompactionStrategy.java   |  4 +-
 .../db/compaction/LeveledManifest.java          | 32 +++++------
 .../SizeTieredCompactionStrategy.java           |  4 +-
 .../SplittingSizeTieredCompactionWriter.java    |  4 +-
 .../db/lifecycle/LifecycleTransaction.java      | 20 +++----
 .../cassandra/db/lifecycle/LogTransaction.java  | 16 +++---
 .../apache/cassandra/db/lifecycle/Tracker.java  |  8 +--
 .../org/apache/cassandra/dht/BootStrapper.java  |  4 +-
 .../org/apache/cassandra/dht/RangeStreamer.java | 12 ++---
 .../cassandra/hadoop/cql3/CqlInputFormat.java   |  4 +-
 .../cassandra/hadoop/cql3/CqlRecordReader.java  |  6 +--
 ...mitedLocalNodeFirstLocalBalancingPolicy.java | 14 ++---
 .../cassandra/hadoop/pig/CqlNativeStorage.java  |  2 +-
 .../apache/cassandra/hints/HintVerbHandler.java |  2 +-
 .../cassandra/hints/HintsDispatchExecutor.java  |  2 +-
 .../cassandra/index/SecondaryIndexManager.java  |  6 +--
 .../index/internal/CassandraIndex.java          |  6 +--
 .../io/sstable/IndexSummaryManager.java         | 10 ++--
 .../apache/cassandra/io/sstable/SSTable.java    |  2 +-
 .../io/sstable/format/SSTableReader.java        | 24 ++++-----
 .../io/sstable/metadata/MetadataSerializer.java |  8 +--
 .../org/apache/cassandra/io/util/FileUtils.java |  8 +--
 .../locator/AbstractReplicationStrategy.java    |  2 +-
 .../locator/NetworkTopologyStrategy.java        |  2 +-
 .../cassandra/locator/PropertyFileSnitch.java   |  6 +--
 .../locator/ReconnectableSnitchHelper.java      |  2 +-
 .../apache/cassandra/locator/TokenMetadata.java |  8 +--
 .../net/IncomingStreamingConnection.java        |  4 +-
 .../cassandra/net/IncomingTcpConnection.java    | 10 ++--
 .../cassandra/net/MessageDeliveryTask.java      |  2 +-
 .../apache/cassandra/net/MessagingService.java  | 20 +++----
 .../cassandra/net/OutboundTcpConnection.java    | 12 ++---
 .../cassandra/net/ResponseVerbHandler.java      |  2 +-
 .../apache/cassandra/schema/SchemaKeyspace.java |  2 +-
 .../cassandra/service/DigestResolver.java       |  8 +--
 .../apache/cassandra/service/GCInspector.java   |  4 +-
 .../cassandra/service/LoadBroadcaster.java      |  4 +-
 .../apache/cassandra/service/ReadCallback.java  |  4 +-
 .../apache/cassandra/service/StorageProxy.java  | 14 ++---
 .../cassandra/thrift/CassandraServer.java       | 56 ++++++++++----------
 .../thrift/CustomTThreadPoolServer.java         |  4 +-
 .../cassandra/thrift/ThriftValidation.java      |  4 +-
 .../org/apache/cassandra/tracing/Tracing.java   |  2 +-
 .../org/apache/cassandra/transport/Message.java |  6 +--
 .../cassandra/triggers/CustomClassLoader.java   |  2 +-
 .../org/apache/cassandra/utils/CLibrary.java    |  2 +-
 .../org/apache/cassandra/utils/Mx4jTool.java    |  4 +-
 .../apache/cassandra/utils/OutputHandler.java   |  2 +-
 .../org/apache/cassandra/utils/TopKSampler.java |  2 +-
 87 files changed, 383 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a2b1b8ab/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------


[8/9] cassandra git commit: Merge branch cassandra-2.2 into cassandra-3.0

Posted by bl...@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/aa60cde3
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/aa60cde3
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/aa60cde3

Branch: refs/heads/trunk
Commit: aa60cde3122a2b512ba4283b2bfd2deaff008004
Parents: 96eb58a 4a849ef
Author: blerer <be...@datastax.com>
Authored: Tue Sep 22 22:04:30 2015 +0200
Committer: blerer <be...@datastax.com>
Committed: Tue Sep 22 22:10:30 2015 +0200

----------------------------------------------------------------------
 NEWS.txt                                        | 16 +++---
 conf/logback.xml                                | 55 ++++++++++++++++---
 .../cassandra/auth/CassandraAuthorizer.java     |  2 +-
 .../cassandra/auth/CassandraRoleManager.java    |  4 +-
 .../cassandra/auth/PasswordAuthenticator.java   |  4 +-
 .../apache/cassandra/auth/PermissionsCache.java |  2 +-
 .../org/apache/cassandra/auth/RolesCache.java   |  2 +-
 .../cassandra/batchlog/BatchlogManager.java     | 10 ++--
 .../batchlog/LegacyBatchlogMigrator.java        |  8 +--
 .../apache/cassandra/cache/AutoSavingCache.java |  8 +--
 .../cassandra/cache/SerializingCache.java       |  2 +-
 .../org/apache/cassandra/client/RingCache.java  |  2 +-
 .../DebuggableScheduledThreadPoolExecutor.java  |  2 +-
 .../DebuggableThreadPoolExecutor.java           |  2 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |  6 +--
 .../cql3/functions/JavaBasedUDFunction.java     |  4 +-
 .../cassandra/cql3/functions/UDFunction.java    |  2 +-
 .../cql3/statements/CreateIndexStatement.java   |  2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  | 44 +++++++--------
 .../apache/cassandra/db/ConsistencyLevel.java   |  6 +--
 .../db/CounterMutationVerbHandler.java          |  2 +-
 .../db/DefinitionsUpdateVerbHandler.java        |  2 +-
 .../org/apache/cassandra/db/Directories.java    |  8 +--
 src/java/org/apache/cassandra/db/Keyspace.java  |  8 +--
 src/java/org/apache/cassandra/db/Memtable.java  | 16 +++---
 .../db/MigrationRequestVerbHandler.java         |  2 +-
 .../cassandra/db/SchemaCheckVerbHandler.java    |  2 +-
 .../cassandra/db/SizeEstimatesRecorder.java     |  6 +--
 .../org/apache/cassandra/db/SystemKeyspace.java |  4 +-
 .../cassandra/db/commitlog/CommitLog.java       |  6 +--
 .../db/commitlog/CommitLogArchiver.java         |  4 +-
 .../db/commitlog/CommitLogReplayer.java         | 22 ++++----
 .../db/commitlog/CommitLogSegmentManager.java   | 14 ++---
 .../db/compaction/CompactionController.java     |  4 +-
 .../db/compaction/CompactionManager.java        | 30 +++++------
 .../compaction/CompactionStrategyManager.java   |  2 +-
 .../cassandra/db/compaction/CompactionTask.java |  8 +--
 .../DateTieredCompactionStrategy.java           | 10 ++--
 .../compaction/LeveledCompactionStrategy.java   |  4 +-
 .../db/compaction/LeveledManifest.java          | 32 +++++------
 .../SizeTieredCompactionStrategy.java           |  4 +-
 .../SplittingSizeTieredCompactionWriter.java    |  4 +-
 .../db/lifecycle/LifecycleTransaction.java      | 20 +++----
 .../cassandra/db/lifecycle/LogTransaction.java  | 16 +++---
 .../apache/cassandra/db/lifecycle/Tracker.java  |  8 +--
 .../org/apache/cassandra/dht/BootStrapper.java  |  4 +-
 .../org/apache/cassandra/dht/RangeStreamer.java | 12 ++---
 .../cassandra/hadoop/cql3/CqlInputFormat.java   |  4 +-
 .../cassandra/hadoop/cql3/CqlRecordReader.java  |  6 +--
 ...mitedLocalNodeFirstLocalBalancingPolicy.java | 14 ++---
 .../cassandra/hadoop/pig/CqlNativeStorage.java  |  2 +-
 .../apache/cassandra/hints/HintVerbHandler.java |  2 +-
 .../cassandra/hints/HintsDispatchExecutor.java  |  2 +-
 .../cassandra/index/SecondaryIndexManager.java  |  6 +--
 .../index/internal/CassandraIndex.java          |  6 +--
 .../io/sstable/IndexSummaryManager.java         | 10 ++--
 .../apache/cassandra/io/sstable/SSTable.java    |  2 +-
 .../io/sstable/format/SSTableReader.java        | 24 ++++-----
 .../io/sstable/metadata/MetadataSerializer.java |  8 +--
 .../org/apache/cassandra/io/util/FileUtils.java |  8 +--
 .../locator/AbstractReplicationStrategy.java    |  2 +-
 .../locator/NetworkTopologyStrategy.java        |  2 +-
 .../cassandra/locator/PropertyFileSnitch.java   |  6 +--
 .../locator/ReconnectableSnitchHelper.java      |  2 +-
 .../apache/cassandra/locator/TokenMetadata.java |  8 +--
 .../net/IncomingStreamingConnection.java        |  4 +-
 .../cassandra/net/IncomingTcpConnection.java    | 10 ++--
 .../cassandra/net/MessageDeliveryTask.java      |  2 +-
 .../apache/cassandra/net/MessagingService.java  | 20 +++----
 .../cassandra/net/OutboundTcpConnection.java    | 12 ++---
 .../cassandra/net/ResponseVerbHandler.java      |  2 +-
 .../apache/cassandra/schema/SchemaKeyspace.java |  2 +-
 .../cassandra/service/DigestResolver.java       |  8 +--
 .../apache/cassandra/service/GCInspector.java   |  4 +-
 .../cassandra/service/LoadBroadcaster.java      |  4 +-
 .../apache/cassandra/service/ReadCallback.java  |  4 +-
 .../apache/cassandra/service/StorageProxy.java  | 14 ++---
 .../cassandra/thrift/CassandraServer.java       | 56 ++++++++++----------
 .../thrift/CustomTThreadPoolServer.java         |  4 +-
 .../cassandra/thrift/ThriftValidation.java      |  4 +-
 .../org/apache/cassandra/tracing/Tracing.java   |  2 +-
 .../org/apache/cassandra/transport/Message.java |  6 +--
 .../cassandra/triggers/CustomClassLoader.java   |  2 +-
 .../org/apache/cassandra/utils/CLibrary.java    |  2 +-
 .../org/apache/cassandra/utils/Mx4jTool.java    |  4 +-
 .../apache/cassandra/utils/OutputHandler.java   |  2 +-
 .../org/apache/cassandra/utils/TopKSampler.java |  2 +-
 87 files changed, 383 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 67398cf,6bd0a77..e4b8663
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -122,6 -27,16 +122,12 @@@ Changed Default
     - commitlog_total_space_in_mb will use the smaller of 8192, and 1/4
       of the total space of the commitlog volume. (Before: always used
       8192)
 -   - Incremental repair is on by default since 2.2.0, run full repairs by
 -     providing the '-full' parameter to nodetool repair.
 -   - Parallel repairs are the default since 2.2.0, run sequential repairs
 -     by providing the '-seq' parameter to nodetool repair.
+    - The following INFO logs were reduced to DEBUG level and will now show
+      on debug.log instead of system.log:
+       - Memtable flushing actions
+       - Commit log replayed files
+       - Compacted sstables
+       - SStable opening (SSTableReader)
  
  New features
  ------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index 934ebaa,0000000..8bc4c26
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@@ -1,554 -1,0 +1,554 @@@
 +/*
 + * 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.batchlog;
 +
 +import java.io.IOException;
 +import java.lang.management.ManagementFactory;
 +import java.net.InetAddress;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.concurrent.*;
 +
 +import javax.management.MBeanServer;
 +import javax.management.ObjectName;
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.collect.*;
 +import com.google.common.util.concurrent.RateLimiter;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.UntypedResultSet;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.marshal.BytesType;
 +import org.apache.cassandra.db.marshal.UUIDType;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.dht.Token;
 +import org.apache.cassandra.exceptions.WriteFailureException;
 +import org.apache.cassandra.exceptions.WriteTimeoutException;
 +import org.apache.cassandra.gms.FailureDetector;
 +import org.apache.cassandra.hints.Hint;
 +import org.apache.cassandra.hints.HintsService;
 +import org.apache.cassandra.io.util.DataInputBuffer;
 +import org.apache.cassandra.io.util.DataOutputBuffer;
 +import org.apache.cassandra.net.MessageIn;
 +import org.apache.cassandra.net.MessageOut;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.service.StorageService;
 +import org.apache.cassandra.service.WriteResponseHandler;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.UUIDGen;
 +
 +import static com.google.common.collect.Iterables.transform;
 +import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 +import static org.apache.cassandra.cql3.QueryProcessor.executeInternalWithPaging;
 +
 +public class BatchlogManager implements BatchlogManagerMBean
 +{
 +    public static final String MBEAN_NAME = "org.apache.cassandra.db:type=BatchlogManager";
 +    private static final long REPLAY_INTERVAL = 10 * 1000; // milliseconds
 +    static final int DEFAULT_PAGE_SIZE = 128;
 +
 +    private static final Logger logger = LoggerFactory.getLogger(BatchlogManager.class);
 +    public static final BatchlogManager instance = new BatchlogManager();
 +
 +    private volatile long totalBatchesReplayed = 0; // no concurrency protection necessary as only written by replay thread.
 +    private volatile UUID lastReplayedUuid = UUIDGen.minTimeUUID(0);
 +
 +    // Single-thread executor service for scheduling and serializing log replay.
 +    private final ScheduledExecutorService batchlogTasks = new DebuggableScheduledThreadPoolExecutor("BatchlogTasks");
 +
 +    public void start()
 +    {
 +        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
 +        try
 +        {
 +            mbs.registerMBean(this, new ObjectName(MBEAN_NAME));
 +        }
 +        catch (Exception e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +
 +        batchlogTasks.scheduleWithFixedDelay(this::replayFailedBatches,
 +                                             StorageService.RING_DELAY,
 +                                             REPLAY_INTERVAL,
 +                                             TimeUnit.MILLISECONDS);
 +    }
 +
 +    public void shutdown() throws InterruptedException
 +    {
 +        batchlogTasks.shutdown();
 +        batchlogTasks.awaitTermination(60, TimeUnit.SECONDS);
 +    }
 +
 +    public static void remove(UUID id)
 +    {
 +        new Mutation(PartitionUpdate.fullPartitionDelete(SystemKeyspace.Batches,
 +                                                         UUIDType.instance.decompose(id),
 +                                                         FBUtilities.timestampMicros(),
 +                                                         FBUtilities.nowInSeconds()))
 +            .apply();
 +    }
 +
 +    public static void store(Batch batch)
 +    {
 +        store(batch, true);
 +    }
 +
 +    public static void store(Batch batch, boolean durableWrites)
 +    {
 +        RowUpdateBuilder builder =
 +            new RowUpdateBuilder(SystemKeyspace.Batches, batch.creationTime, batch.id)
 +                .clustering()
 +                .add("version", MessagingService.current_version);
 +
 +        for (ByteBuffer mutation : batch.encodedMutations)
 +            builder.addListEntry("mutations", mutation);
 +
 +        for (Mutation mutation : batch.decodedMutations)
 +        {
 +            try (DataOutputBuffer buffer = new DataOutputBuffer())
 +            {
 +                Mutation.serializer.serialize(mutation, buffer, MessagingService.current_version);
 +                builder.addListEntry("mutations", buffer.buffer());
 +            }
 +            catch (IOException e)
 +            {
 +                // shouldn't happen
 +                throw new AssertionError(e);
 +            }
 +        }
 +
 +        builder.build().apply(durableWrites);
 +    }
 +
 +    @VisibleForTesting
 +    public int countAllBatches()
 +    {
 +        String query = String.format("SELECT count(*) FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.BATCHES);
 +        UntypedResultSet results = executeInternal(query);
 +        if (results == null || results.isEmpty())
 +            return 0;
 +
 +        return (int) results.one().getLong("count");
 +    }
 +
 +    public long getTotalBatchesReplayed()
 +    {
 +        return totalBatchesReplayed;
 +    }
 +
 +    public void forceBatchlogReplay() throws Exception
 +    {
 +        startBatchlogReplay().get();
 +    }
 +
 +    public Future<?> startBatchlogReplay()
 +    {
 +        // If a replay is already in progress this request will be executed after it completes.
 +        return batchlogTasks.submit(this::replayFailedBatches);
 +    }
 +
 +    void performInitialReplay() throws InterruptedException, ExecutionException
 +    {
 +        // Invokes initial replay. Used for testing only.
 +        batchlogTasks.submit(this::replayFailedBatches).get();
 +    }
 +
 +    private void replayFailedBatches()
 +    {
-         logger.debug("Started replayFailedBatches");
++        logger.trace("Started replayFailedBatches");
 +
 +        // rate limit is in bytes per second. Uses Double.MAX_VALUE if disabled (set to 0 in cassandra.yaml).
 +        // max rate is scaled by the number of nodes in the cluster (same as for HHOM - see CASSANDRA-5272).
 +        int throttleInKB = DatabaseDescriptor.getBatchlogReplayThrottleInKB() / StorageService.instance.getTokenMetadata().getAllEndpoints().size();
 +        RateLimiter rateLimiter = RateLimiter.create(throttleInKB == 0 ? Double.MAX_VALUE : throttleInKB * 1024);
 +
 +        UUID limitUuid = UUIDGen.maxTimeUUID(System.currentTimeMillis() - getBatchlogTimeout());
 +        ColumnFamilyStore store = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHES);
 +        int pageSize = calculatePageSize(store);
 +        // There cannot be any live content where token(id) <= token(lastReplayedUuid) as every processed batch is
 +        // deleted, but the tombstoned content may still be present in the tables. To avoid walking over it we specify
 +        // token(id) > token(lastReplayedUuid) as part of the query.
 +        String query = String.format("SELECT id, mutations, version FROM %s.%s WHERE token(id) > token(?) AND token(id) <= token(?)",
 +                                     SystemKeyspace.NAME,
 +                                     SystemKeyspace.BATCHES);
 +        UntypedResultSet batches = executeInternalWithPaging(query, pageSize, lastReplayedUuid, limitUuid);
 +        processBatchlogEntries(batches, pageSize, rateLimiter);
 +        lastReplayedUuid = limitUuid;
-         logger.debug("Finished replayFailedBatches");
++        logger.trace("Finished replayFailedBatches");
 +    }
 +
 +    // read less rows (batches) per page if they are very large
 +    static int calculatePageSize(ColumnFamilyStore store)
 +    {
 +        double averageRowSize = store.getMeanPartitionSize();
 +        if (averageRowSize <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        return (int) Math.max(1, Math.min(DEFAULT_PAGE_SIZE, 4 * 1024 * 1024 / averageRowSize));
 +    }
 +
 +    private void processBatchlogEntries(UntypedResultSet batches, int pageSize, RateLimiter rateLimiter)
 +    {
 +        int positionInPage = 0;
 +        ArrayList<ReplayingBatch> unfinishedBatches = new ArrayList<>(pageSize);
 +
 +        Set<InetAddress> hintedNodes = new HashSet<>();
 +        Set<UUID> replayedBatches = new HashSet<>();
 +
 +        // Sending out batches for replay without waiting for them, so that one stuck batch doesn't affect others
 +        for (UntypedResultSet.Row row : batches)
 +        {
 +            UUID id = row.getUUID("id");
 +            int version = row.getInt("version");
 +            try
 +            {
 +                ReplayingBatch batch = new ReplayingBatch(id, version, row.getList("mutations", BytesType.instance));
 +                if (batch.replay(rateLimiter, hintedNodes) > 0)
 +                {
 +                    unfinishedBatches.add(batch);
 +                }
 +                else
 +                {
 +                    remove(id); // no write mutations were sent (either expired or all CFs involved truncated).
 +                    ++totalBatchesReplayed;
 +                }
 +            }
 +            catch (IOException e)
 +            {
 +                logger.warn("Skipped batch replay of {} due to {}", id, e);
 +                remove(id);
 +            }
 +
 +            if (++positionInPage == pageSize)
 +            {
 +                // We have reached the end of a batch. To avoid keeping more than a page of mutations in memory,
 +                // finish processing the page before requesting the next row.
 +                finishAndClearBatches(unfinishedBatches, hintedNodes, replayedBatches);
 +                positionInPage = 0;
 +            }
 +        }
 +
 +        finishAndClearBatches(unfinishedBatches, hintedNodes, replayedBatches);
 +
 +        // to preserve batch guarantees, we must ensure that hints (if any) have made it to disk, before deleting the batches
 +        HintsService.instance.flushAndFsyncBlockingly(transform(hintedNodes, StorageService.instance::getHostIdForEndpoint));
 +
 +        // once all generated hints are fsynced, actually delete the batches
 +        replayedBatches.forEach(BatchlogManager::remove);
 +    }
 +
 +    private void finishAndClearBatches(ArrayList<ReplayingBatch> batches, Set<InetAddress> hintedNodes, Set<UUID> replayedBatches)
 +    {
 +        // schedule hints for timed out deliveries
 +        for (ReplayingBatch batch : batches)
 +        {
 +            batch.finish(hintedNodes);
 +            replayedBatches.add(batch.id);
 +        }
 +
 +        totalBatchesReplayed += batches.size();
 +        batches.clear();
 +    }
 +
 +    public static long getBatchlogTimeout()
 +    {
 +        return DatabaseDescriptor.getWriteRpcTimeout() * 2; // enough time for the actual write + BM removal mutation
 +    }
 +
 +    private static class ReplayingBatch
 +    {
 +        private final UUID id;
 +        private final long writtenAt;
 +        private final List<Mutation> mutations;
 +        private final int replayedBytes;
 +
 +        private List<ReplayWriteResponseHandler<Mutation>> replayHandlers;
 +
 +        ReplayingBatch(UUID id, int version, List<ByteBuffer> serializedMutations) throws IOException
 +        {
 +            this.id = id;
 +            this.writtenAt = UUIDGen.unixTimestamp(id);
 +            this.mutations = new ArrayList<>(serializedMutations.size());
 +            this.replayedBytes = addMutations(version, serializedMutations);
 +        }
 +
 +        public int replay(RateLimiter rateLimiter, Set<InetAddress> hintedNodes) throws IOException
 +        {
-             logger.debug("Replaying batch {}", id);
++            logger.trace("Replaying batch {}", id);
 +
 +            if (mutations.isEmpty())
 +                return 0;
 +
 +            int gcgs = gcgs(mutations);
 +            if (TimeUnit.MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds())
 +                return 0;
 +
 +            replayHandlers = sendReplays(mutations, writtenAt, hintedNodes);
 +
 +            rateLimiter.acquire(replayedBytes); // acquire afterwards, to not mess up ttl calculation.
 +
 +            return replayHandlers.size();
 +        }
 +
 +        public void finish(Set<InetAddress> hintedNodes)
 +        {
 +            for (int i = 0; i < replayHandlers.size(); i++)
 +            {
 +                ReplayWriteResponseHandler<Mutation> handler = replayHandlers.get(i);
 +                try
 +                {
 +                    handler.get();
 +                }
 +                catch (WriteTimeoutException|WriteFailureException e)
 +                {
-                     logger.debug("Failed replaying a batched mutation to a node, will write a hint");
-                     logger.debug("Failure was : {}", e.getMessage());
++                    logger.trace("Failed replaying a batched mutation to a node, will write a hint");
++                    logger.trace("Failure was : {}", e.getMessage());
 +                    // writing hints for the rest to hints, starting from i
 +                    writeHintsForUndeliveredEndpoints(i, hintedNodes);
 +                    return;
 +                }
 +            }
 +        }
 +
 +        private int addMutations(int version, List<ByteBuffer> serializedMutations) throws IOException
 +        {
 +            int ret = 0;
 +            for (ByteBuffer serializedMutation : serializedMutations)
 +            {
 +                ret += serializedMutation.remaining();
 +                try (DataInputBuffer in = new DataInputBuffer(serializedMutation, true))
 +                {
 +                    addMutation(Mutation.serializer.deserialize(in, version));
 +                }
 +            }
 +
 +            return ret;
 +        }
 +
 +        // Remove CFs that have been truncated since. writtenAt and SystemTable#getTruncatedAt() both return millis.
 +        // We don't abort the replay entirely b/c this can be considered a success (truncated is same as delivered then
 +        // truncated.
 +        private void addMutation(Mutation mutation)
 +        {
 +            for (UUID cfId : mutation.getColumnFamilyIds())
 +                if (writtenAt <= SystemKeyspace.getTruncatedAt(cfId))
 +                    mutation = mutation.without(cfId);
 +
 +            if (!mutation.isEmpty())
 +                mutations.add(mutation);
 +        }
 +
 +        private void writeHintsForUndeliveredEndpoints(int startFrom, Set<InetAddress> hintedNodes)
 +        {
 +            int gcgs = gcgs(mutations);
 +
 +            // expired
 +            if (TimeUnit.MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds())
 +                return;
 +
 +            for (int i = startFrom; i < replayHandlers.size(); i++)
 +            {
 +                ReplayWriteResponseHandler<Mutation> handler = replayHandlers.get(i);
 +                Mutation undeliveredMutation = mutations.get(i);
 +
 +                if (handler != null)
 +                {
 +                    hintedNodes.addAll(handler.undelivered);
 +                    HintsService.instance.write(transform(handler.undelivered, StorageService.instance::getHostIdForEndpoint),
 +                                                Hint.create(undeliveredMutation, writtenAt));
 +                }
 +            }
 +        }
 +
 +        private static List<ReplayWriteResponseHandler<Mutation>> sendReplays(List<Mutation> mutations,
 +                                                                              long writtenAt,
 +                                                                              Set<InetAddress> hintedNodes)
 +        {
 +            List<ReplayWriteResponseHandler<Mutation>> handlers = new ArrayList<>(mutations.size());
 +            for (Mutation mutation : mutations)
 +            {
 +                ReplayWriteResponseHandler<Mutation> handler = sendSingleReplayMutation(mutation, writtenAt, hintedNodes);
 +                if (handler != null)
 +                    handlers.add(handler);
 +            }
 +            return handlers;
 +        }
 +
 +        /**
 +         * We try to deliver the mutations to the replicas ourselves if they are alive and only resort to writing hints
 +         * when a replica is down or a write request times out.
 +         *
 +         * @return direct delivery handler to wait on or null, if no live nodes found
 +         */
 +        private static ReplayWriteResponseHandler<Mutation> sendSingleReplayMutation(final Mutation mutation,
 +                                                                                     long writtenAt,
 +                                                                                     Set<InetAddress> hintedNodes)
 +        {
 +            Set<InetAddress> liveEndpoints = new HashSet<>();
 +            String ks = mutation.getKeyspaceName();
 +            Token tk = mutation.key().getToken();
 +
 +            for (InetAddress endpoint : Iterables.concat(StorageService.instance.getNaturalEndpoints(ks, tk),
 +                                                         StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, ks)))
 +            {
 +                if (endpoint.equals(FBUtilities.getBroadcastAddress()))
 +                {
 +                    mutation.apply();
 +                }
 +                else if (FailureDetector.instance.isAlive(endpoint))
 +                {
 +                    liveEndpoints.add(endpoint); // will try delivering directly instead of writing a hint.
 +                }
 +                else
 +                {
 +                    hintedNodes.add(endpoint);
 +                    HintsService.instance.write(StorageService.instance.getHostIdForEndpoint(endpoint),
 +                                                Hint.create(mutation, writtenAt));
 +                }
 +            }
 +
 +            if (liveEndpoints.isEmpty())
 +                return null;
 +
 +            ReplayWriteResponseHandler<Mutation> handler = new ReplayWriteResponseHandler<>(liveEndpoints);
 +            MessageOut<Mutation> message = mutation.createMessage();
 +            for (InetAddress endpoint : liveEndpoints)
 +                MessagingService.instance().sendRR(message, endpoint, handler, false);
 +            return handler;
 +        }
 +
 +        private static int gcgs(Collection<Mutation> mutations)
 +        {
 +            int gcgs = Integer.MAX_VALUE;
 +            for (Mutation mutation : mutations)
 +                gcgs = Math.min(gcgs, mutation.smallestGCGS());
 +            return gcgs;
 +        }
 +
 +        /**
 +         * A wrapper of WriteResponseHandler that stores the addresses of the endpoints from
 +         * which we did not receive a successful reply.
 +         */
 +        private static class ReplayWriteResponseHandler<T> extends WriteResponseHandler<T>
 +        {
 +            private final Set<InetAddress> undelivered = Collections.newSetFromMap(new ConcurrentHashMap<>());
 +
 +            ReplayWriteResponseHandler(Collection<InetAddress> writeEndpoints)
 +            {
 +                super(writeEndpoints, Collections.<InetAddress>emptySet(), null, null, null, WriteType.UNLOGGED_BATCH);
 +                undelivered.addAll(writeEndpoints);
 +            }
 +
 +            @Override
 +            protected int totalBlockFor()
 +            {
 +                return this.naturalEndpoints.size();
 +            }
 +
 +            @Override
 +            public void response(MessageIn<T> m)
 +            {
 +                boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddress() : m.from);
 +                assert removed;
 +                super.response(m);
 +            }
 +        }
 +    }
 +
 +    public static class EndpointFilter
 +    {
 +        private final String localRack;
 +        private final Multimap<String, InetAddress> endpoints;
 +
 +        public EndpointFilter(String localRack, Multimap<String, InetAddress> endpoints)
 +        {
 +            this.localRack = localRack;
 +            this.endpoints = endpoints;
 +        }
 +
 +        /**
 +         * @return list of candidates for batchlog hosting. If possible these will be two nodes from different racks.
 +         */
 +        public Collection<InetAddress> filter()
 +        {
 +            // special case for single-node data centers
 +            if (endpoints.values().size() == 1)
 +                return endpoints.values();
 +
 +            // strip out dead endpoints and localhost
 +            ListMultimap<String, InetAddress> validated = ArrayListMultimap.create();
 +            for (Map.Entry<String, InetAddress> entry : endpoints.entries())
 +                if (isValid(entry.getValue()))
 +                    validated.put(entry.getKey(), entry.getValue());
 +
 +            if (validated.size() <= 2)
 +                return validated.values();
 +
 +            if (validated.size() - validated.get(localRack).size() >= 2)
 +            {
 +                // we have enough endpoints in other racks
 +                validated.removeAll(localRack);
 +            }
 +
 +            if (validated.keySet().size() == 1)
 +            {
 +                // we have only 1 `other` rack
 +                Collection<InetAddress> otherRack = Iterables.getOnlyElement(validated.asMap().values());
 +                return Lists.newArrayList(Iterables.limit(otherRack, 2));
 +            }
 +
 +            // randomize which racks we pick from if more than 2 remaining
 +            Collection<String> racks;
 +            if (validated.keySet().size() == 2)
 +            {
 +                racks = validated.keySet();
 +            }
 +            else
 +            {
 +                racks = Lists.newArrayList(validated.keySet());
 +                Collections.shuffle((List<String>) racks);
 +            }
 +
 +            // grab a random member of up to two racks
 +            List<InetAddress> result = new ArrayList<>(2);
 +            for (String rack : Iterables.limit(racks, 2))
 +            {
 +                List<InetAddress> rackMembers = validated.get(rack);
 +                result.add(rackMembers.get(getRandomInt(rackMembers.size())));
 +            }
 +
 +            return result;
 +        }
 +
 +        @VisibleForTesting
 +        protected boolean isValid(InetAddress input)
 +        {
 +            return !input.equals(FBUtilities.getBroadcastAddress()) && FailureDetector.instance.isAlive(input);
 +        }
 +
 +        @VisibleForTesting
 +        protected int getRandomInt(int bound)
 +        {
 +            return ThreadLocalRandom.current().nextInt(bound);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/batchlog/LegacyBatchlogMigrator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/batchlog/LegacyBatchlogMigrator.java
index 13ff81a,0000000..dd19f19
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/batchlog/LegacyBatchlogMigrator.java
+++ b/src/java/org/apache/cassandra/batchlog/LegacyBatchlogMigrator.java
@@@ -1,196 -1,0 +1,196 @@@
 +/*
 + * 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.batchlog;
 +
 +import java.io.IOException;
 +import java.net.InetAddress;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.cql3.QueryProcessor;
 +import org.apache.cassandra.cql3.UntypedResultSet;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.marshal.UUIDType;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.exceptions.WriteFailureException;
 +import org.apache.cassandra.exceptions.WriteTimeoutException;
 +import org.apache.cassandra.io.util.DataInputBuffer;
 +import org.apache.cassandra.io.util.DataOutputBuffer;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.service.AbstractWriteResponseHandler;
 +import org.apache.cassandra.service.WriteResponseHandler;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.UUIDGen;
 +
 +public final class LegacyBatchlogMigrator
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LegacyBatchlogMigrator.class);
 +
 +    private LegacyBatchlogMigrator()
 +    {
 +        // static class
 +    }
 +
 +    @SuppressWarnings("deprecation")
 +    public static void migrate()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.LEGACY_BATCHLOG);
 +
 +        // nothing to migrate
 +        if (store.isEmpty())
 +            return;
 +
 +        logger.info("Migrating legacy batchlog to new storage");
 +
 +        int convertedBatches = 0;
 +        String query = String.format("SELECT id, data, written_at, version FROM %s.%s",
 +                                     SystemKeyspace.NAME,
 +                                     SystemKeyspace.LEGACY_BATCHLOG);
 +
 +        int pageSize = BatchlogManager.calculatePageSize(store);
 +
 +        UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, pageSize);
 +        for (UntypedResultSet.Row row : rows)
 +        {
 +            if (apply(row, convertedBatches))
 +                convertedBatches++;
 +        }
 +
 +        if (convertedBatches > 0)
 +            Keyspace.openAndGetStore(SystemKeyspace.LegacyBatchlog).truncateBlocking();
 +    }
 +
 +    @SuppressWarnings("deprecation")
 +    public static boolean isLegacyBatchlogMutation(Mutation mutation)
 +    {
 +        return mutation.getKeyspaceName().equals(SystemKeyspace.NAME)
 +            && mutation.getPartitionUpdate(SystemKeyspace.LegacyBatchlog.cfId) != null;
 +    }
 +
 +    @SuppressWarnings("deprecation")
 +    public static void handleLegacyMutation(Mutation mutation)
 +    {
 +        PartitionUpdate update = mutation.getPartitionUpdate(SystemKeyspace.LegacyBatchlog.cfId);
-         logger.debug("Applying legacy batchlog mutation {}", update);
++        logger.trace("Applying legacy batchlog mutation {}", update);
 +        update.forEach(row -> apply(UntypedResultSet.Row.fromInternalRow(update.metadata(), update.partitionKey(), row), -1));
 +    }
 +
 +    private static boolean apply(UntypedResultSet.Row row, long counter)
 +    {
 +        UUID id = row.getUUID("id");
 +        long timestamp = id.version() == 1 ? UUIDGen.unixTimestamp(id) : row.getLong("written_at");
 +        int version = row.has("version") ? row.getInt("version") : MessagingService.VERSION_12;
 +
 +        if (id.version() != 1)
 +            id = UUIDGen.getTimeUUID(timestamp, counter);
 +
-         logger.debug("Converting mutation at {}", timestamp);
++        logger.trace("Converting mutation at {}", timestamp);
 +
 +        try (DataInputBuffer in = new DataInputBuffer(row.getBytes("data"), false))
 +        {
 +            int numMutations = in.readInt();
 +            List<Mutation> mutations = new ArrayList<>(numMutations);
 +            for (int i = 0; i < numMutations; i++)
 +                mutations.add(Mutation.serializer.deserialize(in, version));
 +
 +            BatchlogManager.store(Batch.createLocal(id, TimeUnit.MILLISECONDS.toMicros(timestamp), mutations));
 +            return true;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.error("Failed to convert mutation {} at timestamp {}", id, timestamp, t);
 +            return false;
 +        }
 +    }
 +
 +    public static void syncWriteToBatchlog(WriteResponseHandler<?> handler, Batch batch, Collection<InetAddress> endpoints)
 +    throws WriteTimeoutException, WriteFailureException
 +    {
 +        for (InetAddress target : endpoints)
 +        {
-             logger.debug("Sending legacy batchlog store request {} to {} for {} mutations", batch.id, target, batch.size());
++            logger.trace("Sending legacy batchlog store request {} to {} for {} mutations", batch.id, target, batch.size());
 +
 +            int targetVersion = MessagingService.instance().getVersion(target);
 +            MessagingService.instance().sendRR(getStoreMutation(batch, targetVersion).createMessage(MessagingService.Verb.MUTATION),
 +                                               target,
 +                                               handler,
 +                                               false);
 +        }
 +    }
 +
 +    public static void asyncRemoveFromBatchlog(Collection<InetAddress> endpoints, UUID uuid)
 +    {
 +        AbstractWriteResponseHandler<IMutation> handler = new WriteResponseHandler<>(endpoints,
 +                                                                                     Collections.<InetAddress>emptyList(),
 +                                                                                     ConsistencyLevel.ANY,
 +                                                                                     Keyspace.open(SystemKeyspace.NAME),
 +                                                                                     null,
 +                                                                                     WriteType.SIMPLE);
 +        Mutation mutation = getRemoveMutation(uuid);
 +
 +        for (InetAddress target : endpoints)
 +        {
-             logger.debug("Sending legacy batchlog remove request {} to {}", uuid, target);
++            logger.trace("Sending legacy batchlog remove request {} to {}", uuid, target);
 +            MessagingService.instance().sendRR(mutation.createMessage(MessagingService.Verb.MUTATION), target, handler, false);
 +        }
 +    }
 +
 +    static void store(Batch batch, int version)
 +    {
 +        getStoreMutation(batch, version).apply();
 +    }
 +
 +    @SuppressWarnings("deprecation")
 +    static Mutation getStoreMutation(Batch batch, int version)
 +    {
 +        return new RowUpdateBuilder(SystemKeyspace.LegacyBatchlog, batch.creationTime, batch.id)
 +               .clustering()
 +               .add("written_at", new Date(batch.creationTime / 1000))
 +               .add("data", getSerializedMutations(version, batch.decodedMutations))
 +               .add("version", version)
 +               .build();
 +    }
 +
 +    @SuppressWarnings("deprecation")
 +    private static Mutation getRemoveMutation(UUID uuid)
 +    {
 +        return new Mutation(PartitionUpdate.fullPartitionDelete(SystemKeyspace.LegacyBatchlog,
 +                                                                UUIDType.instance.decompose(uuid),
 +                                                                FBUtilities.timestampMicros(),
 +                                                                FBUtilities.nowInSeconds()));
 +    }
 +
 +    private static ByteBuffer getSerializedMutations(int version, Collection<Mutation> mutations)
 +    {
 +        try (DataOutputBuffer buf = new DataOutputBuffer())
 +        {
 +            buf.writeInt(mutations.size());
 +            for (Mutation mutation : mutations)
 +                Mutation.serializer.serialize(mutation, buf, version);
 +            return buf.buffer();
 +        }
 +        catch (IOException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/cache/AutoSavingCache.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/cache/SerializingCache.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
index 2aafeb9,0000000..1db13e3
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
@@@ -1,628 -1,0 +1,628 @@@
 +/*
 + * 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.cql3.functions;
 +
 +import java.io.ByteArrayOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.lang.invoke.MethodHandle;
 +import java.lang.invoke.MethodHandles;
 +import java.lang.invoke.MethodType;
 +import java.lang.reflect.InvocationTargetException;
 +import java.net.*;
 +import java.nio.ByteBuffer;
 +import java.security.*;
 +import java.security.cert.Certificate;
 +import java.util.*;
 +import java.util.concurrent.ConcurrentHashMap;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.ThreadLocalRandom;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import com.google.common.io.ByteStreams;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.datastax.driver.core.DataType;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.eclipse.jdt.core.compiler.IProblem;
 +import org.eclipse.jdt.internal.compiler.*;
 +import org.eclipse.jdt.internal.compiler.Compiler;
 +import org.eclipse.jdt.internal.compiler.classfmt.ClassFileReader;
 +import org.eclipse.jdt.internal.compiler.classfmt.ClassFormatException;
 +import org.eclipse.jdt.internal.compiler.env.ICompilationUnit;
 +import org.eclipse.jdt.internal.compiler.env.INameEnvironment;
 +import org.eclipse.jdt.internal.compiler.env.NameEnvironmentAnswer;
 +import org.eclipse.jdt.internal.compiler.impl.CompilerOptions;
 +import org.eclipse.jdt.internal.compiler.problem.DefaultProblemFactory;
 +
 +final class JavaBasedUDFunction extends UDFunction
 +{
 +    private static final String BASE_PACKAGE = "org.apache.cassandra.cql3.udf.gen";
 +
 +    static final Logger logger = LoggerFactory.getLogger(JavaBasedUDFunction.class);
 +
 +    private static final AtomicInteger classSequence = new AtomicInteger();
 +
 +    // use a JVM standard ExecutorService as DebuggableThreadPoolExecutor references internal
 +    // classes, which triggers AccessControlException from the UDF sandbox
 +    private static final UDFExecutorService executor =
 +        new UDFExecutorService(new NamedThreadFactory("UserDefinedFunctions",
 +                                                      Thread.MIN_PRIORITY,
 +                                                      udfClassLoader,
 +                                                      new SecurityThreadGroup("UserDefinedFunctions", null)),
 +                               "userfunction");
 +
 +    private static final EcjTargetClassLoader targetClassLoader = new EcjTargetClassLoader();
 +
 +    private static final UDFByteCodeVerifier udfByteCodeVerifier = new UDFByteCodeVerifier();
 +
 +    private static final ProtectionDomain protectionDomain;
 +
 +    private static final IErrorHandlingPolicy errorHandlingPolicy = DefaultErrorHandlingPolicies.proceedWithAllProblems();
 +    private static final IProblemFactory problemFactory = new DefaultProblemFactory(Locale.ENGLISH);
 +    private static final CompilerOptions compilerOptions;
 +
 +    /**
 +     * Poor man's template - just a text file splitted at '#' chars.
 +     * Each string at an even index is a constant string (just copied),
 +     * each string at an odd index is an 'instruction'.
 +     */
 +    private static final String[] javaSourceTemplate;
 +
 +    static
 +    {
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/Class", "forName");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/Class", "getClassLoader");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/Class", "getResource");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/Class", "getResourceAsStream");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "clearAssertionStatus");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getResource");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getResourceAsStream");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getResources");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getSystemClassLoader");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getSystemResource");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getSystemResourceAsStream");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getSystemResources");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "loadClass");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "setClassAssertionStatus");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "setDefaultAssertionStatus");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "setPackageAssertionStatus");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/nio/ByteBuffer", "allocateDirect");
 +
 +        Map<String, String> settings = new HashMap<>();
 +        settings.put(CompilerOptions.OPTION_LineNumberAttribute,
 +                     CompilerOptions.GENERATE);
 +        settings.put(CompilerOptions.OPTION_SourceFileAttribute,
 +                     CompilerOptions.DISABLED);
 +        settings.put(CompilerOptions.OPTION_ReportDeprecation,
 +                     CompilerOptions.IGNORE);
 +        settings.put(CompilerOptions.OPTION_Source,
 +                     CompilerOptions.VERSION_1_8);
 +        settings.put(CompilerOptions.OPTION_TargetPlatform,
 +                     CompilerOptions.VERSION_1_8);
 +
 +        compilerOptions = new CompilerOptions(settings);
 +        compilerOptions.parseLiteralExpressionsAsConstants = true;
 +
 +        try (InputStream input = JavaBasedUDFunction.class.getResource("JavaSourceUDF.txt").openConnection().getInputStream())
 +        {
 +            ByteArrayOutputStream output = new ByteArrayOutputStream();
 +            FBUtilities.copy(input, output, Long.MAX_VALUE);
 +            String template = output.toString();
 +
 +            StringTokenizer st = new StringTokenizer(template, "#");
 +            javaSourceTemplate = new String[st.countTokens()];
 +            for (int i = 0; st.hasMoreElements(); i++)
 +                javaSourceTemplate[i] = st.nextToken();
 +        }
 +        catch (IOException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +
 +        CodeSource codeSource;
 +        try
 +        {
 +            codeSource = new CodeSource(new URL("udf", "localhost", 0, "/java", new URLStreamHandler()
 +            {
 +                protected URLConnection openConnection(URL u)
 +                {
 +                    return null;
 +                }
 +            }), (Certificate[])null);
 +        }
 +        catch (MalformedURLException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +
 +        protectionDomain = new ProtectionDomain(codeSource, ThreadAwareSecurityManager.noPermissions, targetClassLoader, null);
 +    }
 +
 +    private final JavaUDF javaUDF;
 +
 +    JavaBasedUDFunction(FunctionName name, List<ColumnIdentifier> argNames, List<AbstractType<?>> argTypes,
 +                        AbstractType<?> returnType, boolean calledOnNullInput, String body)
 +    {
 +        super(name, argNames, argTypes, UDHelper.driverTypes(argTypes),
 +              returnType, UDHelper.driverType(returnType), calledOnNullInput, "java", body);
 +
 +        // javaParamTypes is just the Java representation for argTypes resp. argDataTypes
 +        Class<?>[] javaParamTypes = UDHelper.javaTypes(argDataTypes, calledOnNullInput);
 +        // javaReturnType is just the Java representation for returnType resp. returnDataType
 +        Class<?> javaReturnType = UDHelper.asJavaClass(returnDataType);
 +
 +        // put each UDF in a separate package to prevent cross-UDF code access
 +        String pkgName = BASE_PACKAGE + '.' + generateClassName(name, 'p');
 +        String clsName = generateClassName(name, 'C');
 +
 +        String executeInternalName = generateClassName(name, 'x');
 +
 +        StringBuilder javaSourceBuilder = new StringBuilder();
 +        int lineOffset = 1;
 +        for (int i = 0; i < javaSourceTemplate.length; i++)
 +        {
 +            String s = javaSourceTemplate[i];
 +
 +            // strings at odd indexes are 'instructions'
 +            if ((i & 1) == 1)
 +            {
 +                switch (s)
 +                {
 +                    case "package_name":
 +                        s = pkgName;
 +                        break;
 +                    case "class_name":
 +                        s = clsName;
 +                        break;
 +                    case "body":
 +                        lineOffset = countNewlines(javaSourceBuilder);
 +                        s = body;
 +                        break;
 +                    case "arguments":
 +                        s = generateArguments(javaParamTypes, argNames);
 +                        break;
 +                    case "argument_list":
 +                        s = generateArgumentList(javaParamTypes, argNames);
 +                        break;
 +                    case "return_type":
 +                        s = javaSourceName(javaReturnType);
 +                        break;
 +                    case "execute_internal_name":
 +                        s = executeInternalName;
 +                        break;
 +                }
 +            }
 +
 +            javaSourceBuilder.append(s);
 +        }
 +
 +        String targetClassName = pkgName + '.' + clsName;
 +
 +        String javaSource = javaSourceBuilder.toString();
 +
-         logger.debug("Compiling Java source UDF '{}' as class '{}' using source:\n{}", name, targetClassName, javaSource);
++        logger.trace("Compiling Java source UDF '{}' as class '{}' using source:\n{}", name, targetClassName, javaSource);
 +
 +        try
 +        {
 +            EcjCompilationUnit compilationUnit = new EcjCompilationUnit(javaSource, targetClassName);
 +
 +            org.eclipse.jdt.internal.compiler.Compiler compiler = new Compiler(compilationUnit,
 +                                                                               errorHandlingPolicy,
 +                                                                               compilerOptions,
 +                                                                               compilationUnit,
 +                                                                               problemFactory);
 +            compiler.compile(new ICompilationUnit[]{ compilationUnit });
 +
 +            if (compilationUnit.problemList != null && !compilationUnit.problemList.isEmpty())
 +            {
 +                boolean fullSource = false;
 +                StringBuilder problems = new StringBuilder();
 +                for (IProblem problem : compilationUnit.problemList)
 +                {
 +                    long ln = problem.getSourceLineNumber() - lineOffset;
 +                    if (ln < 1L)
 +                    {
 +                        if (problem.isError())
 +                        {
 +                            // if generated source around UDF source provided by the user is buggy,
 +                            // this code is appended.
 +                            problems.append("GENERATED SOURCE ERROR: line ")
 +                                    .append(problem.getSourceLineNumber())
 +                                    .append(" (in generated source): ")
 +                                    .append(problem.getMessage())
 +                                    .append('\n');
 +                            fullSource = true;
 +                        }
 +                    }
 +                    else
 +                    {
 +                        problems.append("Line ")
 +                                .append(Long.toString(ln))
 +                                .append(": ")
 +                                .append(problem.getMessage())
 +                                .append('\n');
 +                    }
 +                }
 +
 +                if (fullSource)
 +                    throw new InvalidRequestException("Java source compilation failed:\n" + problems + "\n generated source:\n" + javaSource);
 +                else
 +                    throw new InvalidRequestException("Java source compilation failed:\n" + problems);
 +            }
 +
 +            // Verify the UDF bytecode against use of probably dangerous code
 +            Set<String> errors = udfByteCodeVerifier.verify(targetClassLoader.classData(targetClassName));
 +            String validDeclare = "not allowed method declared: " + executeInternalName + '(';
 +            String validCall = "call to " + targetClassName.replace('.', '/') + '.' + executeInternalName + "()";
 +            for (Iterator<String> i = errors.iterator(); i.hasNext();)
 +            {
 +                String error = i.next();
 +                // we generate a random name of the private, internal execute method, which is detected by the byte-code verifier
 +                if (error.startsWith(validDeclare) || error.equals(validCall))
 +                {
 +                    i.remove();
 +                }
 +            }
 +            if (!errors.isEmpty())
 +                throw new InvalidRequestException("Java UDF validation failed: " + errors);
 +
 +            // Load the class and create a new instance of it
 +            Thread thread = Thread.currentThread();
 +            ClassLoader orig = thread.getContextClassLoader();
 +            try
 +            {
 +                thread.setContextClassLoader(UDFunction.udfClassLoader);
 +                // Execute UDF intiialization from UDF class loader
 +
 +                Class cls = Class.forName(targetClassName, false, targetClassLoader);
 +
 +                if (cls.getDeclaredMethods().length != 2 || cls.getDeclaredConstructors().length != 1)
 +                    throw new InvalidRequestException("Check your source to not define additional Java methods or constructors");
 +                MethodType methodType = MethodType.methodType(void.class)
 +                                                  .appendParameterTypes(DataType.class, DataType[].class);
 +                MethodHandle ctor = MethodHandles.lookup().findConstructor(cls, methodType);
 +                this.javaUDF = (JavaUDF) ctor.invokeWithArguments(returnDataType, argDataTypes);
 +            }
 +            finally
 +            {
 +                thread.setContextClassLoader(orig);
 +            }
 +        }
 +        catch (InvocationTargetException e)
 +        {
 +            // in case of an ITE, use the cause
 +            throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e.getCause()));
 +        }
 +        catch (VirtualMachineError e)
 +        {
 +            throw e;
 +        }
 +        catch (Throwable e)
 +        {
 +            throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e));
 +        }
 +    }
 +
 +    protected ExecutorService executor()
 +    {
 +        return executor;
 +    }
 +
 +    protected ByteBuffer executeUserDefined(int protocolVersion, List<ByteBuffer> params)
 +    {
 +        return javaUDF.executeImpl(protocolVersion, params);
 +    }
 +
 +
 +    private static int countNewlines(StringBuilder javaSource)
 +    {
 +        int ln = 0;
 +        for (int i = 0; i < javaSource.length(); i++)
 +            if (javaSource.charAt(i) == '\n')
 +                ln++;
 +        return ln;
 +    }
 +
 +    private static String generateClassName(FunctionName name, char prefix)
 +    {
 +        String qualifiedName = name.toString();
 +
 +        StringBuilder sb = new StringBuilder(qualifiedName.length() + 10);
 +        sb.append(prefix);
 +        for (int i = 0; i < qualifiedName.length(); i++)
 +        {
 +            char c = qualifiedName.charAt(i);
 +            if (Character.isJavaIdentifierPart(c))
 +                sb.append(c);
 +            else
 +                sb.append(Integer.toHexString(((short)c)&0xffff));
 +        }
 +        sb.append('_')
 +          .append(ThreadLocalRandom.current().nextInt() & 0xffffff)
 +          .append('_')
 +          .append(classSequence.incrementAndGet());
 +        return sb.toString();
 +    }
 +
 +    private static String javaSourceName(Class<?> type)
 +    {
 +        String n = type.getName();
 +        return n.startsWith("java.lang.") ? type.getSimpleName() : n;
 +    }
 +
 +    private static String generateArgumentList(Class<?>[] paramTypes, List<ColumnIdentifier> argNames)
 +    {
 +        // initial builder size can just be a guess (prevent temp object allocations)
 +        StringBuilder code = new StringBuilder(32 * paramTypes.length);
 +        for (int i = 0; i < paramTypes.length; i++)
 +        {
 +            if (i > 0)
 +                code.append(", ");
 +            code.append(javaSourceName(paramTypes[i]))
 +                .append(' ')
 +                .append(argNames.get(i));
 +        }
 +        return code.toString();
 +    }
 +
 +    private static String generateArguments(Class<?>[] paramTypes, List<ColumnIdentifier> argNames)
 +    {
 +        StringBuilder code = new StringBuilder(64 * paramTypes.length);
 +        for (int i = 0; i < paramTypes.length; i++)
 +        {
 +            if (i > 0)
 +                code.append(",\n");
 +
-             if (logger.isDebugEnabled())
++            if (logger.isTraceEnabled())
 +                code.append("            /* parameter '").append(argNames.get(i)).append("' */\n");
 +
 +            code
 +                // cast to Java type
 +                .append("            (").append(javaSourceName(paramTypes[i])).append(") ")
 +                // generate object representation of input parameter (call UDFunction.compose)
 +                .append(composeMethod(paramTypes[i])).append("(protocolVersion, ").append(i).append(", params.get(").append(i).append("))");
 +        }
 +        return code.toString();
 +    }
 +
 +    private static String composeMethod(Class<?> type)
 +    {
 +        return (type.isPrimitive()) ? ("super.compose_" + type.getName()) : "super.compose";
 +    }
 +
 +    // Java source UDFs are a very simple compilation task, which allows us to let one class implement
 +    // all interfaces required by ECJ.
 +    static final class EcjCompilationUnit implements ICompilationUnit, ICompilerRequestor, INameEnvironment
 +    {
 +        List<IProblem> problemList;
 +        private final String className;
 +        private final char[] sourceCode;
 +
 +        EcjCompilationUnit(String sourceCode, String className)
 +        {
 +            this.className = className;
 +            this.sourceCode = sourceCode.toCharArray();
 +        }
 +
 +        // ICompilationUnit
 +
 +        @Override
 +        public char[] getFileName()
 +        {
 +            return sourceCode;
 +        }
 +
 +        @Override
 +        public char[] getContents()
 +        {
 +            return sourceCode;
 +        }
 +
 +        @Override
 +        public char[] getMainTypeName()
 +        {
 +            int dot = className.lastIndexOf('.');
 +            return ((dot > 0) ? className.substring(dot + 1) : className).toCharArray();
 +        }
 +
 +        @Override
 +        public char[][] getPackageName()
 +        {
 +            StringTokenizer izer = new StringTokenizer(className, ".");
 +            char[][] result = new char[izer.countTokens() - 1][];
 +            for (int i = 0; i < result.length; i++)
 +                result[i] = izer.nextToken().toCharArray();
 +            return result;
 +        }
 +
 +        @Override
 +        public boolean ignoreOptionalProblems()
 +        {
 +            return false;
 +        }
 +
 +        // ICompilerRequestor
 +
 +        @Override
 +        public void acceptResult(CompilationResult result)
 +        {
 +            if (result.hasErrors())
 +            {
 +                IProblem[] problems = result.getProblems();
 +                if (problemList == null)
 +                    problemList = new ArrayList<>(problems.length);
 +                Collections.addAll(problemList, problems);
 +            }
 +            else
 +            {
 +                ClassFile[] classFiles = result.getClassFiles();
 +                for (ClassFile classFile : classFiles)
 +                    targetClassLoader.addClass(className, classFile.getBytes());
 +            }
 +        }
 +
 +        // INameEnvironment
 +
 +        @Override
 +        public NameEnvironmentAnswer findType(char[][] compoundTypeName)
 +        {
 +            StringBuilder result = new StringBuilder();
 +            for (int i = 0; i < compoundTypeName.length; i++)
 +            {
 +                if (i > 0)
 +                    result.append('.');
 +                result.append(compoundTypeName[i]);
 +            }
 +            return findType(result.toString());
 +        }
 +
 +        @Override
 +        public NameEnvironmentAnswer findType(char[] typeName, char[][] packageName)
 +        {
 +            StringBuilder result = new StringBuilder();
 +            int i = 0;
 +            for (; i < packageName.length; i++)
 +            {
 +                if (i > 0)
 +                    result.append('.');
 +                result.append(packageName[i]);
 +            }
 +            if (i > 0)
 +                result.append('.');
 +            result.append(typeName);
 +            return findType(result.toString());
 +        }
 +
 +        private NameEnvironmentAnswer findType(String className)
 +        {
 +            if (className.equals(this.className))
 +            {
 +                return new NameEnvironmentAnswer(this, null);
 +            }
 +
 +            String resourceName = className.replace('.', '/') + ".class";
 +
 +            try (InputStream is = UDFunction.udfClassLoader.getResourceAsStream(resourceName))
 +            {
 +                if (is != null)
 +                {
 +                    byte[] classBytes = ByteStreams.toByteArray(is);
 +                    char[] fileName = className.toCharArray();
 +                    ClassFileReader classFileReader = new ClassFileReader(classBytes, fileName, true);
 +                    return new NameEnvironmentAnswer(classFileReader, null);
 +                }
 +            }
 +            catch (IOException | ClassFormatException exc)
 +            {
 +                throw new RuntimeException(exc);
 +            }
 +            return null;
 +        }
 +
 +        private boolean isPackage(String result)
 +        {
 +            if (result.equals(this.className))
 +                return false;
 +            String resourceName = result.replace('.', '/') + ".class";
 +            try (InputStream is = UDFunction.udfClassLoader.getResourceAsStream(resourceName))
 +            {
 +                return is == null;
 +            }
 +            catch (IOException e)
 +            {
 +                // we are here, since close on is failed. That means it was not null
 +                return false;
 +            }
 +        }
 +
 +        @Override
 +        public boolean isPackage(char[][] parentPackageName, char[] packageName)
 +        {
 +            StringBuilder result = new StringBuilder();
 +            int i = 0;
 +            if (parentPackageName != null)
 +                for (; i < parentPackageName.length; i++)
 +                {
 +                    if (i > 0)
 +                        result.append('.');
 +                    result.append(parentPackageName[i]);
 +                }
 +
 +            if (Character.isUpperCase(packageName[0]) && !isPackage(result.toString()))
 +                return false;
 +            if (i > 0)
 +                result.append('.');
 +            result.append(packageName);
 +
 +            return isPackage(result.toString());
 +        }
 +
 +        @Override
 +        public void cleanup()
 +        {
 +        }
 +    }
 +
 +    static final class EcjTargetClassLoader extends SecureClassLoader
 +    {
 +        EcjTargetClassLoader()
 +        {
 +            super(UDFunction.udfClassLoader);
 +        }
 +
 +        // This map is usually empty.
 +        // It only contains data *during* UDF compilation but not during runtime.
 +        //
 +        // addClass() is invoked by ECJ after successful compilation of the generated Java source.
 +        // loadClass(targetClassName) is invoked by buildUDF() after ECJ returned from successful compilation.
 +        //
 +        private final Map<String, byte[]> classes = new ConcurrentHashMap<>();
 +
 +        void addClass(String className, byte[] classData)
 +        {
 +            classes.put(className, classData);
 +        }
 +
 +        byte[] classData(String className)
 +        {
 +            return classes.get(className);
 +        }
 +
 +        protected Class<?> findClass(String name) throws ClassNotFoundException
 +        {
 +            // remove the class binary - it's only used once - so it's wasting heap
 +            byte[] classData = classes.remove(name);
 +
 +            if (classData != null)
 +                return defineClass(name, classData, 0, classData.length, protectionDomain);
 +
 +            return getParent().loadClass(name);
 +        }
 +
 +        protected PermissionCollection getPermissions(CodeSource codesource)
 +        {
 +            return ThreadAwareSecurityManager.noPermissions;
 +        }
 +    }}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index e21d8af,1e5cea6..a07852d
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@@ -263,160 -143,11 +263,160 @@@ public abstract class UDFunction extend
              return null;
  
          long tStart = System.nanoTime();
 -        ByteBuffer result = executeUserDefined(protocolVersion, parameters);
 -        Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
 -        return result;
 +        parameters = makeEmptyParametersNull(parameters);
 +
 +        try
 +        {
 +            // Using async UDF execution is expensive (adds about 100us overhead per invocation on a Core-i7 MBPr).
 +            ByteBuffer result = DatabaseDescriptor.enableUserDefinedFunctionsThreads()
 +                                ? executeAsync(protocolVersion, parameters)
 +                                : executeUserDefined(protocolVersion, parameters);
 +
 +            Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
 +            return result;
 +        }
 +        catch (InvalidRequestException e)
 +        {
 +            throw e;
 +        }
 +        catch (Throwable t)
 +        {
-             logger.debug("Invocation of user-defined function '{}' failed", this, t);
++            logger.trace("Invocation of user-defined function '{}' failed", this, t);
 +            if (t instanceof VirtualMachineError)
 +                throw (VirtualMachineError) t;
 +            throw FunctionExecutionException.create(this, t);
 +        }
 +    }
 +
 +    public static void assertUdfsEnabled(String language)
 +    {
 +        if (!DatabaseDescriptor.enableUserDefinedFunctions())
 +            throw new InvalidRequestException("User-defined functions are disabled in cassandra.yaml - set enable_user_defined_functions=true to enable");
 +        if (!"java".equalsIgnoreCase(language) && !DatabaseDescriptor.enableScriptedUserDefinedFunctions())
 +            throw new InvalidRequestException("Scripted user-defined functions are disabled in cassandra.yaml - set enable_scripted_user_defined_functions=true to enable if you are aware of the security risks");
 +    }
 +
 +    private static final class ThreadIdAndCpuTime extends CompletableFuture<Object>
 +    {
 +        long threadId;
 +        long cpuTime;
 +
 +        ThreadIdAndCpuTime()
 +        {
 +            // Looks weird?
 +            // This call "just" links this class to java.lang.management - otherwise UDFs (script UDFs) might fail due to
 +            //      java.security.AccessControlException: access denied: ("java.lang.RuntimePermission" "accessClassInPackage.java.lang.management")
 +            // because class loading would be deferred until setup() is executed - but setup() is called with
 +            // limited privileges.
 +            threadMXBean.getCurrentThreadCpuTime();
 +            //
 +            // Get the TypeCodec stuff in Java Driver initialized.
 +            UDHelper.codecRegistry.codecFor(DataType.inet()).format(InetAddress.getLoopbackAddress());
 +            UDHelper.codecRegistry.codecFor(DataType.ascii()).format("");
 +        }
 +
 +        void setup()
 +        {
 +            this.threadId = Thread.currentThread().getId();
 +            this.cpuTime = threadMXBean.getCurrentThreadCpuTime();
 +            complete(null);
 +        }
 +    }
 +
 +    private ByteBuffer executeAsync(int protocolVersion, List<ByteBuffer> parameters)
 +    {
 +        ThreadIdAndCpuTime threadIdAndCpuTime = new ThreadIdAndCpuTime();
 +
 +        Future<ByteBuffer> future = executor().submit(() -> {
 +            threadIdAndCpuTime.setup();
 +            return executeUserDefined(protocolVersion, parameters);
 +        });
 +
 +        try
 +        {
 +            if (DatabaseDescriptor.getUserDefinedFunctionWarnTimeout() > 0)
 +                try
 +                {
 +                    return future.get(DatabaseDescriptor.getUserDefinedFunctionWarnTimeout(), TimeUnit.MILLISECONDS);
 +                }
 +                catch (TimeoutException e)
 +                {
 +
 +                    // log and emit a warning that UDF execution took long
 +                    String warn = String.format("User defined function %s ran longer than %dms", this, DatabaseDescriptor.getUserDefinedFunctionWarnTimeout());
 +                    logger.warn(warn);
 +                    ClientWarn.warn(warn);
 +                }
 +
 +            // retry with difference of warn-timeout to fail-timeout
 +            return future.get(DatabaseDescriptor.getUserDefinedFunctionFailTimeout() - DatabaseDescriptor.getUserDefinedFunctionWarnTimeout(), TimeUnit.MILLISECONDS);
 +        }
 +        catch (InterruptedException e)
 +        {
 +            Thread.currentThread().interrupt();
 +            throw new RuntimeException(e);
 +        }
 +        catch (ExecutionException e)
 +        {
 +            Throwable c = e.getCause();
 +            if (c instanceof RuntimeException)
 +                throw (RuntimeException) c;
 +            throw new RuntimeException(c);
 +        }
 +        catch (TimeoutException e)
 +        {
 +            // retry a last time with the difference of UDF-fail-timeout to consumed CPU time (just in case execution hit a badly timed GC)
 +            try
 +            {
 +                //The threadIdAndCpuTime shouldn't take a long time to be set so this should return immediately
 +                threadIdAndCpuTime.get(1, TimeUnit.SECONDS);
 +
 +                long cpuTimeMillis = threadMXBean.getThreadCpuTime(threadIdAndCpuTime.threadId) - threadIdAndCpuTime.cpuTime;
 +                cpuTimeMillis /= 1000000L;
 +
 +                return future.get(Math.max(DatabaseDescriptor.getUserDefinedFunctionFailTimeout() - cpuTimeMillis, 0L),
 +                                  TimeUnit.MILLISECONDS);
 +            }
 +            catch (InterruptedException e1)
 +            {
 +                Thread.currentThread().interrupt();
 +                throw new RuntimeException(e);
 +            }
 +            catch (ExecutionException e1)
 +            {
 +                Throwable c = e.getCause();
 +                if (c instanceof RuntimeException)
 +                    throw (RuntimeException) c;
 +                throw new RuntimeException(c);
 +            }
 +            catch (TimeoutException e1)
 +            {
 +                TimeoutException cause = new TimeoutException(String.format("User defined function %s ran longer than %dms%s",
 +                                                                            this,
 +                                                                            DatabaseDescriptor.getUserDefinedFunctionFailTimeout(),
 +                                                                            DatabaseDescriptor.getUserFunctionTimeoutPolicy() == Config.UserFunctionTimeoutPolicy.ignore
 +                                                                            ? "" : " - will stop Cassandra VM"));
 +                FunctionExecutionException fe = FunctionExecutionException.create(this, cause);
 +                JVMStabilityInspector.userFunctionTimeout(cause);
 +                throw fe;
 +            }
 +        }
      }
  
 +    private List<ByteBuffer> makeEmptyParametersNull(List<ByteBuffer> parameters)
 +    {
 +        List<ByteBuffer> r = new ArrayList<>(parameters.size());
 +        for (int i = 0; i < parameters.size(); i++)
 +        {
 +            ByteBuffer param = parameters.get(i);
 +            r.add(UDHelper.isNullOrEmpty(argTypes.get(i), param)
 +                  ? null : param);
 +        }
 +        return r;
 +    }
 +
 +    protected abstract ExecutorService executor();
 +
      public boolean isCallableWrtNullable(List<ByteBuffer> parameters)
      {
          if (!calledOnNullInput)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 0735103,edc092d..d11d2c5
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@@ -220,22 -186,11 +220,22 @@@ public class CreateIndexStatement exten
          }
          else
          {
 -            cd.setIndexType(IndexType.KEYS, Collections.<String, String>emptyMap());
 +            indexOptions = Collections.emptyMap();
 +            kind = cfm.isCompound() ? IndexMetadata.Kind.COMPOSITES : IndexMetadata.Kind.KEYS;
          }
  
 -        cd.setIndexName(indexName);
 -        cfm.addDefaultIndexNames();
 +        IndexMetadata index = IndexMetadata.fromIndexTargets(cfm, targets, acceptedName, kind, indexOptions);
 +
 +        // check to disallow creation of an index which duplicates an existing one in all but name
 +        Optional<IndexMetadata> existingIndex = Iterables.tryFind(cfm.getIndexes(), existing -> existing.equalsWithoutName(index));
 +        if (existingIndex.isPresent())
 +            throw new InvalidRequestException(String.format("Index %s is a duplicate of existing index %s",
 +                                                            index.name,
 +                                                            existingIndex.get().name));
 +
-         logger.debug("Updating index definition for {}", indexName);
++        logger.trace("Updating index definition for {}", indexName);
 +        cfm.indexes(cfm.getIndexes().with(index));
 +
          MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly);
          return true;
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index a9a8f80,4b418b4..062eb0a
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -244,10 -214,10 +244,10 @@@ public class ColumnFamilyStore implemen
  
      void scheduleFlush()
      {
 -        int period = metadata.getMemtableFlushPeriod();
 +        int period = metadata.params.memtableFlushPeriodInMs;
          if (period > 0)
          {
-             logger.debug("scheduling flush in {} ms", period);
+             logger.trace("scheduling flush in {} ms", period);
              WrappedRunnable runnable = new WrappedRunnable()
              {
                  protected void runMayThrow() throws Exception
@@@ -422,7 -422,7 +422,7 @@@
              {
                  throw new RuntimeException(e);
              }
-             logger.debug("retryPolicy for {} is {}", name, this.metadata.params.speculativeRetry);
 -            logger.trace("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
++            logger.trace("retryPolicy for {} is {}", name, this.metadata.params.speculativeRetry);
              latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
              {
                  public void run()
@@@ -564,14 -543,36 +564,14 @@@
      {
          Directories directories = new Directories(metadata);
  
 -        // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
 +         // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
          clearEphemeralSnapshots(directories);
  
-         logger.debug("Removing temporary or obsoleted files from unfinished operations for table", metadata.cfName);
 -        // remove any left-behind SSTables from failed/stalled streaming
 -        FileFilter filter = new FileFilter()
 -        {
 -            public boolean accept(File pathname)
 -            {
 -                return pathname.getPath().endsWith(StreamLockfile.FILE_EXT);
 -            }
 -        };
 -        for (File dir : directories.getCFDirectories())
 -        {
 -            File[] lockfiles = dir.listFiles(filter);
 -            // lock files can be null if I/O error happens
 -            if (lockfiles == null || lockfiles.length == 0)
 -                continue;
 -            logger.info("Removing SSTables from failed streaming session. Found {} files to cleanup.", lockfiles.length);
 -
 -            for (File lockfile : lockfiles)
 -            {
 -                StreamLockfile streamLockfile = new StreamLockfile(lockfile);
 -                streamLockfile.cleanup();
 -                streamLockfile.delete();
 -            }
 -        }
 -
 -        logger.trace("Removing compacted SSTable files from {} (see http://wiki.apache.org/cassandra/MemtableSSTable)", metadata.cfName);
++        logger.trace("Removing temporary or obsoleted files from unfinished operations for table", metadata.cfName);
 +        LifecycleTransaction.removeUnfinishedLeftovers(metadata);
  
-         logger.debug("Further extra check for orphan sstable files for {}", metadata.cfName);
 -        for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister().list().entrySet())
++        logger.trace("Further extra check for orphan sstable files for {}", metadata.cfName);
 +        for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister(Directories.OnTxnErr.IGNORE).list().entrySet())
          {
              Descriptor desc = sstableFiles.getKey();
              Set<Component> components = sstableFiles.getValue();
@@@ -813,16 -904,19 +813,16 @@@
          onHeapTotal += memtable.getAllocator().onHeap().owns();
          offHeapTotal += memtable.getAllocator().offHeap().owns();
  
 -        for (SecondaryIndex index : indexManager.getIndexes())
 +        for (ColumnFamilyStore indexCfs : indexManager.getAllIndexColumnFamilyStores())
          {
 -            if (index.getIndexCfs() != null)
 -            {
 -                MemtableAllocator allocator = index.getIndexCfs().getTracker().getView().getCurrentMemtable().getAllocator();
 -                onHeapRatio += allocator.onHeap().ownershipRatio();
 -                offHeapRatio += allocator.offHeap().ownershipRatio();
 -                onHeapTotal += allocator.onHeap().owns();
 -                offHeapTotal += allocator.offHeap().owns();
 -            }
 +            MemtableAllocator allocator = indexCfs.getTracker().getView().getCurrentMemtable().getAllocator();
 +            onHeapRatio += allocator.onHeap().ownershipRatio();
 +            offHeapRatio += allocator.offHeap().ownershipRatio();
 +            onHeapTotal += allocator.onHeap().owns();
 +            offHeapTotal += allocator.offHeap().owns();
          }
  
-         logger.info("Enqueuing flush of {}: {}", name, String.format("%d (%.0f%%) on-heap, %d (%.0f%%) off-heap",
+         logger.debug("Enqueuing flush of {}: {}", name, String.format("%d (%.0f%%) on-heap, %d (%.0f%%) off-heap",
                                                                       onHeapTotal, onHeapRatio * 100, offHeapTotal, offHeapRatio * 100));
      }
  
@@@ -1152,9 -1341,9 +1152,9 @@@
       * @return sstables whose key range overlaps with that of the given sstables, not including itself.
       * (The given sstables may or may not overlap with each other.)
       */
 -    public Collection<SSTableReader> getOverlappingSSTables(Iterable<SSTableReader> sstables)
 +    public Collection<SSTableReader> getOverlappingSSTables(SSTableSet sstableSet, Iterable<SSTableReader> sstables)
      {
-         logger.debug("Checking for sstables overlapping {}", sstables);
+         logger.trace("Checking for sstables overlapping {}", sstables);
  
          // a normal compaction won't ever have an empty sstables list, but we create a skeleton
          // compaction controller for streaming, and that passes an empty list.
@@@ -1636,10 -2409,10 +1636,10 @@@
                  SSTableReader sstable = active.get(entries.getKey().generation);
                  if (sstable == null || !refs.tryRef(sstable))
                  {
-                     if (logger.isDebugEnabled())
-                         logger.debug("using snapshot sstable {}", entries.getKey());
+                     if (logger.isTraceEnabled())
+                         logger.trace("using snapshot sstable {}", entries.getKey());
                      // open without tracking hotness
 -                    sstable = SSTableReader.open(entries.getKey(), entries.getValue(), metadata, partitioner, true, false);
 +                    sstable = SSTableReader.open(entries.getKey(), entries.getValue(), metadata, true, false);
                      refs.tryRef(sstable);
                      // release the self ref as we never add the snapshot sstable to DataTracker where it is otherwise released
                      sstable.selfRef().release();
@@@ -1854,9 -2634,9 +1854,9 @@@
          // beginning if we restart before they [the CL segments] are discarded for
          // normal reasons post-truncate.  To prevent this, we store truncation
          // position in the System keyspace.
-         logger.debug("truncating {}", name);
+         logger.trace("truncating {}", name);
  
 -        if (keyspace.getMetadata().durableWrites || DatabaseDescriptor.isAutoSnapshot())
 +        if (keyspace.getMetadata().params.durableWrites || DatabaseDescriptor.isAutoSnapshot())
          {
              // flush the CF being truncated before forcing the new segment
              forceBlockingFlush();
@@@ -1887,47 -2670,30 +1887,47 @@@
  
                  ReplayPosition replayAfter = discardSSTables(truncatedAt);
  
 -                for (SecondaryIndex index : indexManager.getIndexes())
 -                    index.truncateBlocking(truncatedAt);
 +                indexManager.truncateAllIndexesBlocking(truncatedAt);
 +
 +                viewManager.truncateBlocking(truncatedAt);
  
                  SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
-                 logger.debug("cleaning out row cache");
+                 logger.trace("cleaning out row cache");
                  invalidateCaches();
              }
          };
  
 -        runWithCompactionsDisabled(Executors.callable(truncateRunnable), true);
 +        runWithCompactionsDisabled(Executors.callable(truncateRunnable), true, true);
-         logger.debug("truncate complete");
+         logger.trace("truncate complete");
      }
  
 -    public <V> V runWithCompactionsDisabled(Callable<V> callable, boolean interruptValidation)
 +    /**
 +     * Drops current memtable without flushing to disk. This should only be called when truncating a column family which is not durable.
 +     */
 +    public void dumpMemtable()
 +    {
 +        synchronized (data)
 +        {
 +            final Flush flush = new Flush(true);
 +            flushExecutor.execute(flush);
 +            postFlushExecutor.submit(flush.postFlush);
 +        }
 +    }
 +
 +    public <V> V runWithCompactionsDisabled(Callable<V> callable, boolean interruptValidation, boolean interruptViews)
      {
          // synchronize so that concurrent invocations don't re-enable compactions partway through unexpectedly,
          // and so we only run one major compaction at a time
          synchronized (this)
          {
-             logger.debug("Cancelling in-progress compactions for {}", metadata.cfName);
+             logger.trace("Cancelling in-progress compactions for {}", metadata.cfName);
  
 -            Iterable<ColumnFamilyStore> selfWithIndexes = concatWithIndexes();
 -            for (ColumnFamilyStore cfs : selfWithIndexes)
 -                cfs.getCompactionStrategy().pause();
 +            Iterable<ColumnFamilyStore> selfWithAuxiliaryCfs = interruptViews
 +                                                               ? Iterables.concat(concatWithIndexes(), viewManager.allViewsCfs())
 +                                                               : concatWithIndexes();
 +
 +            for (ColumnFamilyStore cfs : selfWithAuxiliaryCfs)
 +                cfs.getCompactionStrategyManager().pause();
              try
              {
                  // interrupt in-progress compactions


[4/9] cassandra git commit: Merge branch cassandra-2.2 into cassandra-3.0

Posted by bl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/service/DigestResolver.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/DigestResolver.java
index 572df6f,0000000..62b4538
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/service/DigestResolver.java
+++ b/src/java/org/apache/cassandra/service/DigestResolver.java
@@@ -1,98 -1,0 +1,98 @@@
 +/*
 + * 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.service;
 +
 +import java.nio.ByteBuffer;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.partitions.PartitionIterator;
 +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
 +import org.apache.cassandra.net.MessageIn;
 +
 +public class DigestResolver extends ResponseResolver
 +{
 +    private volatile ReadResponse dataResponse;
 +
 +    public DigestResolver(Keyspace keyspace, ReadCommand command, ConsistencyLevel consistency, int maxResponseCount)
 +    {
 +        super(keyspace, command, consistency, maxResponseCount);
 +    }
 +
 +    @Override
 +    public void preprocess(MessageIn<ReadResponse> message)
 +    {
 +        super.preprocess(message);
 +        if (dataResponse == null && !message.payload.isDigestResponse())
 +            dataResponse = message.payload;
 +    }
 +
 +    /**
 +     * Special case of resolve() so that CL.ONE reads never throw DigestMismatchException in the foreground
 +     */
 +    public PartitionIterator getData()
 +    {
 +        assert isDataPresent();
 +        return UnfilteredPartitionIterators.filter(dataResponse.makeIterator(command.metadata(), command), command.nowInSec());
 +    }
 +
 +    /*
 +     * This method handles two different scenarios:
 +     *
 +     * a) we're handling the initial read of data from the closest replica + digests
 +     *    from the rest. In this case we check the digests against each other,
 +     *    throw an exception if there is a mismatch, otherwise return the data row.
 +     *
 +     * b) we're checking additional digests that arrived after the minimum to handle
 +     *    the requested ConsistencyLevel, i.e. asynchronous read repair check
 +     */
 +    public PartitionIterator resolve() throws DigestMismatchException
 +    {
 +        if (responses.size() == 1)
 +            return getData();
 +
-         if (logger.isDebugEnabled())
-             logger.debug("resolving {} responses", responses.size());
++        if (logger.isTraceEnabled())
++            logger.trace("resolving {} responses", responses.size());
 +
 +        long start = System.nanoTime();
 +
 +        // validate digests against each other; throw immediately on mismatch.
 +        ByteBuffer digest = null;
 +        for (MessageIn<ReadResponse> message : responses)
 +        {
 +            ReadResponse response = message.payload;
 +
 +            ByteBuffer newDigest = response.digest(command.metadata(), command);
 +            if (digest == null)
 +                digest = newDigest;
 +            else if (!digest.equals(newDigest))
 +                // rely on the fact that only single partition queries use digests
 +                throw new DigestMismatchException(((SinglePartitionReadCommand)command).partitionKey(), digest, newDigest);
 +        }
 +
-         if (logger.isDebugEnabled())
-             logger.debug("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
++        if (logger.isTraceEnabled())
++            logger.trace("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 +
 +        return UnfilteredPartitionIterators.filter(dataResponse.makeIterator(command.metadata(), command), command.nowInSec());
 +    }
 +
 +    public boolean isDataPresent()
 +    {
 +        return dataResponse != null;
 +    }
 +}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/service/ReadCallback.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/ReadCallback.java
index 8b1ef32,145679d..8747004
--- a/src/java/org/apache/cassandra/service/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/ReadCallback.java
@@@ -108,46 -102,31 +108,46 @@@ public class ReadCallback implements IA
          }
      }
  
 -    public TResolved get() throws ReadFailureException, ReadTimeoutException, DigestMismatchException
 +    public void awaitResults() throws ReadFailureException, ReadTimeoutException
      {
 -        if (!await(command.getTimeout(), TimeUnit.MILLISECONDS))
 +        boolean signaled = await(command.getTimeout(), TimeUnit.MILLISECONDS);
 +        boolean failed = blockfor + failures > endpoints.size();
 +        if (signaled && !failed)
 +            return;
 +
 +        if (Tracing.isTracing())
          {
 -            // Same as for writes, see AbstractWriteResponseHandler
 -            ReadTimeoutException ex = new ReadTimeoutException(consistencyLevel, received, blockfor, resolver.isDataPresent());
 -            Tracing.trace("Read timeout: {}", ex.toString());
 -            if (logger.isTraceEnabled())
 -                logger.trace("Read timeout: {}", ex.toString());
 -            throw ex;
 +            String gotData = received > 0 ? (resolver.isDataPresent() ? " (including data)" : " (only digests)") : "";
 +            Tracing.trace("{}; received {} of {} responses{}", new Object[]{ (failed ? "Failed" : "Timed out"), received, blockfor, gotData });
          }
 -
 -        if (blockfor + failures > endpoints.size())
 +        else if (logger.isDebugEnabled())
          {
 -            ReadFailureException ex = new ReadFailureException(consistencyLevel, received, failures, blockfor, resolver.isDataPresent());
 -
 -            if (logger.isTraceEnabled())
 -                logger.trace("Read failure: {}", ex.toString());
 -            throw ex;
 +            String gotData = received > 0 ? (resolver.isDataPresent() ? " (including data)" : " (only digests)") : "";
 +            logger.debug("{}; received {} of {} responses{}", new Object[]{ (failed ? "Failed" : "Timed out"), received, blockfor, gotData });
          }
  
 -        return blockfor == 1 ? resolver.getData() : resolver.resolve();
 +        // Same as for writes, see AbstractWriteResponseHandler
 +        throw failed
 +            ? new ReadFailureException(consistencyLevel, received, failures, blockfor, resolver.isDataPresent())
 +            : new ReadTimeoutException(consistencyLevel, received, blockfor, resolver.isDataPresent());
      }
  
 -    public void response(MessageIn<TMessage> message)
 +    public PartitionIterator get() throws ReadFailureException, ReadTimeoutException, DigestMismatchException
 +    {
 +        awaitResults();
 +
 +        PartitionIterator result = blockfor == 1 ? resolver.getData() : resolver.resolve();
-         if (logger.isDebugEnabled())
-             logger.debug("Read: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
++        if (logger.isTraceEnabled())
++            logger.trace("Read: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 +        return result;
 +    }
 +
 +    public int blockFor()
 +    {
 +        return blockfor;
 +    }
 +
 +    public void response(MessageIn<ReadResponse> message)
      {
          resolver.preprocess(message);
          int n = waitingFor(message.from)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageProxy.java
index d209af6,af56c3a..5c94f08
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@@ -819,86 -703,59 +819,86 @@@ public class StorageProxy implements St
          }
      }
  
 -    private static void syncWriteToBatchlog(Collection<Mutation> mutations, Collection<InetAddress> endpoints, UUID uuid)
 +    public static boolean canDoLocalRequest(InetAddress replica)
 +    {
 +        return replica.equals(FBUtilities.getBroadcastAddress());
 +    }
 +
 +    private static void syncWriteToBatchlog(Collection<Mutation> mutations, BatchlogEndpoints endpoints, UUID uuid)
 +    throws WriteTimeoutException, WriteFailureException
 +    {
 +        WriteResponseHandler<?> handler = new WriteResponseHandler<>(endpoints.all,
 +                                                                     Collections.<InetAddress>emptyList(),
 +                                                                     endpoints.all.size() == 1 ? ConsistencyLevel.ONE : ConsistencyLevel.TWO,
 +                                                                     Keyspace.open(SystemKeyspace.NAME),
 +                                                                     null,
 +                                                                     WriteType.BATCH_LOG);
 +
 +        Batch batch = Batch.createLocal(uuid, FBUtilities.timestampMicros(), mutations);
 +
 +        if (!endpoints.current.isEmpty())
 +            syncWriteToBatchlog(handler, batch, endpoints.current);
 +
 +        if (!endpoints.legacy.isEmpty())
 +            LegacyBatchlogMigrator.syncWriteToBatchlog(handler, batch, endpoints.legacy);
 +
 +        handler.get();
 +    }
 +
 +    private static void syncWriteToBatchlog(WriteResponseHandler<?> handler, Batch batch, Collection<InetAddress> endpoints)
      throws WriteTimeoutException, WriteFailureException
      {
 -        AbstractWriteResponseHandler<IMutation> handler = new WriteResponseHandler<>(endpoints,
 -                                                                        Collections.<InetAddress>emptyList(),
 -                                                                        ConsistencyLevel.ONE,
 -                                                                        Keyspace.open(SystemKeyspace.NAME),
 -                                                                        null,
 -                                                                        WriteType.BATCH_LOG);
 +        MessageOut<Batch> message = new MessageOut<>(MessagingService.Verb.BATCH_STORE, batch, Batch.serializer);
  
 -        MessageOut<Mutation> message = BatchlogManager.getBatchlogMutationFor(mutations, uuid, MessagingService.current_version)
 -                                                      .createMessage();
          for (InetAddress target : endpoints)
          {
-             logger.debug("Sending batchlog store request {} to {} for {} mutations", batch.id, target, batch.size());
 -            int targetVersion = MessagingService.instance().getVersion(target);
 -            if (target.equals(FBUtilities.getBroadcastAddress()) && OPTIMIZE_LOCAL_REQUESTS)
 -            {
 -                insertLocal(message.payload, handler);
 -            }
 -            else if (targetVersion == MessagingService.current_version)
 -            {
 -                MessagingService.instance().sendRR(message, target, handler, false);
 -            }
++            logger.trace("Sending batchlog store request {} to {} for {} mutations", batch.id, target, batch.size());
 +
 +            if (canDoLocalRequest(target))
 +                performLocally(Stage.MUTATION, () -> BatchlogManager.store(batch), handler);
              else
 -            {
 -                MessagingService.instance().sendRR(BatchlogManager.getBatchlogMutationFor(mutations, uuid, targetVersion)
 -                                                                  .createMessage(),
 -                                                   target,
 -                                                   handler,
 -                                                   false);
 -            }
 +                MessagingService.instance().sendRR(message, target, handler);
          }
 +    }
  
 -        handler.get();
 +    private static void asyncRemoveFromBatchlog(BatchlogEndpoints endpoints, UUID uuid)
 +    {
 +        if (!endpoints.current.isEmpty())
 +            asyncRemoveFromBatchlog(endpoints.current, uuid);
 +
 +        if (!endpoints.legacy.isEmpty())
 +            LegacyBatchlogMigrator.asyncRemoveFromBatchlog(endpoints.legacy, uuid);
      }
  
      private static void asyncRemoveFromBatchlog(Collection<InetAddress> endpoints, UUID uuid)
      {
 -        AbstractWriteResponseHandler<IMutation> handler = new WriteResponseHandler<>(endpoints,
 -                                                                        Collections.<InetAddress>emptyList(),
 -                                                                        ConsistencyLevel.ANY,
 -                                                                        Keyspace.open(SystemKeyspace.NAME),
 -                                                                        null,
 -                                                                        WriteType.SIMPLE);
 -        Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(uuid));
 -        mutation.delete(SystemKeyspace.BATCHLOG, FBUtilities.timestampMicros());
 -        MessageOut<Mutation> message = mutation.createMessage();
 +        MessageOut<UUID> message = new MessageOut<>(MessagingService.Verb.BATCH_REMOVE, uuid, UUIDSerializer.serializer);
          for (InetAddress target : endpoints)
          {
-             if (logger.isDebugEnabled())
-                 logger.debug("Sending batchlog remove request {} to {}", uuid, target);
 -            if (target.equals(FBUtilities.getBroadcastAddress()) && OPTIMIZE_LOCAL_REQUESTS)
 -                insertLocal(message.payload, handler);
++            if (logger.isTraceEnabled())
++                logger.trace("Sending batchlog remove request {} to {}", uuid, target);
 +
 +            if (canDoLocalRequest(target))
 +                performLocally(Stage.MUTATION, () -> BatchlogManager.remove(uuid));
              else
 -                MessagingService.instance().sendRR(message, target, handler, false);
 +                MessagingService.instance().sendOneWay(message, target);
 +        }
 +    }
 +
 +    private static void asyncWriteBatchedMutations(List<WriteResponseHandlerWrapper> wrappers, String localDataCenter, Stage stage)
 +    {
 +        for (WriteResponseHandlerWrapper wrapper : wrappers)
 +        {
 +            Iterable<InetAddress> endpoints = Iterables.concat(wrapper.handler.naturalEndpoints, wrapper.handler.pendingEndpoints);
 +
 +            try
 +            {
 +                sendToHintedEndpoints(wrapper.mutation, endpoints, wrapper.handler, localDataCenter, stage);
 +            }
 +            catch (OverloadedException | WriteTimeoutException e)
 +            {
 +                wrapper.handler.onFailure(FBUtilities.getBroadcastAddress());
 +            }
          }
      }
  
@@@ -1522,144 -1347,218 +1522,144 @@@
       * 4. If the digests (if any) match the data return the data
       * 5. else carry out read repair by getting data from all the nodes.
       */
 -    private static List<Row> fetchRows(List<ReadCommand> initialCommands, ConsistencyLevel consistencyLevel)
 +    private static PartitionIterator fetchRows(List<SinglePartitionReadCommand<?>> commands, ConsistencyLevel consistencyLevel)
      throws UnavailableException, ReadFailureException, ReadTimeoutException
      {
 -        List<Row> rows = new ArrayList<>(initialCommands.size());
 -        // (avoid allocating a new list in the common case of nothing-to-retry)
 -        List<ReadCommand> commandsToRetry = Collections.emptyList();
 +        int cmdCount = commands.size();
  
 -        do
 -        {
 -            List<ReadCommand> commands = commandsToRetry.isEmpty() ? initialCommands : commandsToRetry;
 -            AbstractReadExecutor[] readExecutors = new AbstractReadExecutor[commands.size()];
 +        SinglePartitionReadLifecycle[] reads = new SinglePartitionReadLifecycle[cmdCount];
 +        for (int i = 0; i < cmdCount; i++)
 +            reads[i] = new SinglePartitionReadLifecycle(commands.get(i), consistencyLevel);
  
 -            if (!commandsToRetry.isEmpty())
 -                Tracing.trace("Retrying {} commands", commandsToRetry.size());
 +        for (int i = 0; i < cmdCount; i++)
 +            reads[i].doInitialQueries();
  
 -            // send out read requests
 -            for (int i = 0; i < commands.size(); i++)
 -            {
 -                ReadCommand command = commands.get(i);
 -                assert !command.isDigestQuery();
 +        for (int i = 0; i < cmdCount; i++)
 +            reads[i].maybeTryAdditionalReplicas();
  
 -                AbstractReadExecutor exec = AbstractReadExecutor.getReadExecutor(command, consistencyLevel);
 -                exec.executeAsync();
 -                readExecutors[i] = exec;
 -            }
 +        for (int i = 0; i < cmdCount; i++)
 +            reads[i].awaitResultsAndRetryOnDigestMismatch();
  
 -            for (AbstractReadExecutor exec : readExecutors)
 -                exec.maybeTryAdditionalReplicas();
 +        for (int i = 0; i < cmdCount; i++)
 +            if (!reads[i].isDone())
 +                reads[i].maybeAwaitFullDataRead();
  
 -            // read results and make a second pass for any digest mismatches
 -            List<ReadCommand> repairCommands = null;
 -            List<ReadCallback<ReadResponse, Row>> repairResponseHandlers = null;
 -            for (AbstractReadExecutor exec: readExecutors)
 -            {
 -                try
 -                {
 -                    Row row = exec.get();
 -                    if (row != null)
 -                    {
 -                        row = exec.command.maybeTrim(row);
 -                        rows.add(row);
 -                    }
 +        List<PartitionIterator> results = new ArrayList<>(cmdCount);
 +        for (int i = 0; i < cmdCount; i++)
 +        {
 +            assert reads[i].isDone();
 +            results.add(reads[i].getResult());
 +        }
  
 -                    if (logger.isTraceEnabled())
 -                        logger.trace("Read: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - exec.handler.start));
 -                }
 -                catch (ReadTimeoutException|ReadFailureException ex)
 -                {
 -                    int blockFor = consistencyLevel.blockFor(Keyspace.open(exec.command.getKeyspace()));
 -                    int responseCount = exec.handler.getReceivedCount();
 -                    String gotData = responseCount > 0
 -                                   ? exec.resolver.isDataPresent() ? " (including data)" : " (only digests)"
 -                                   : "";
 -
 -                    boolean isTimeout = ex instanceof ReadTimeoutException;
 -                    if (Tracing.isTracing())
 -                    {
 -                        Tracing.trace("{}; received {} of {} responses{}",
 -                                      isTimeout ? "Timed out" : "Failed", responseCount, blockFor, gotData);
 -                    }
 -                    else if (logger.isDebugEnabled())
 -                    {
 -                        logger.debug("Read {}; received {} of {} responses{}", (isTimeout ? "timeout" : "failure"), responseCount, blockFor, gotData);
 -                    }
 -                    throw ex;
 -                }
 -                catch (DigestMismatchException ex)
 -                {
 -                    Tracing.trace("Digest mismatch: {}", ex);
 +        return PartitionIterators.concat(results);
 +    }
  
 -                    ReadRepairMetrics.repairedBlocking.mark();
 +    private static class SinglePartitionReadLifecycle
 +    {
 +        private final SinglePartitionReadCommand<?> command;
 +        private final AbstractReadExecutor executor;
 +        private final ConsistencyLevel consistency;
  
 -                    // Do a full data read to resolve the correct response (and repair node that need be)
 -                    RowDataResolver resolver = new RowDataResolver(exec.command.ksName, exec.command.key, exec.command.filter(), exec.command.timestamp, exec.handler.endpoints.size());
 -                    ReadCallback<ReadResponse, Row> repairHandler = new ReadCallback<>(resolver,
 -                                                                                       ConsistencyLevel.ALL,
 -                                                                                       exec.getContactedReplicas().size(),
 -                                                                                       exec.command,
 -                                                                                       Keyspace.open(exec.command.getKeyspace()),
 -                                                                                       exec.handler.endpoints);
 +        private PartitionIterator result;
 +        private ReadCallback repairHandler;
  
 -                    if (repairCommands == null)
 -                    {
 -                        repairCommands = new ArrayList<>();
 -                        repairResponseHandlers = new ArrayList<>();
 -                    }
 -                    repairCommands.add(exec.command);
 -                    repairResponseHandlers.add(repairHandler);
 +        SinglePartitionReadLifecycle(SinglePartitionReadCommand<?> command, ConsistencyLevel consistency)
 +        {
 +            this.command = command;
 +            this.executor = AbstractReadExecutor.getReadExecutor(command, consistency);
 +            this.consistency = consistency;
 +        }
  
 -                    MessageOut<ReadCommand> message = exec.command.createMessage();
 -                    for (InetAddress endpoint : exec.getContactedReplicas())
 -                    {
 -                        Tracing.trace("Enqueuing full data read to {}", endpoint);
 -                        MessagingService.instance().sendRRWithFailure(message, endpoint, repairHandler);
 -                    }
 -                }
 -            }
 +        boolean isDone()
 +        {
 +            return result != null;
 +        }
  
 -            commandsToRetry.clear();
 +        void doInitialQueries()
 +        {
 +            executor.executeAsync();
 +        }
  
 -            // read the results for the digest mismatch retries
 -            if (repairResponseHandlers != null)
 -            {
 -                for (int i = 0; i < repairCommands.size(); i++)
 -                {
 -                    ReadCommand command = repairCommands.get(i);
 -                    ReadCallback<ReadResponse, Row> handler = repairResponseHandlers.get(i);
 +        void maybeTryAdditionalReplicas()
 +        {
 +            executor.maybeTryAdditionalReplicas();
 +        }
  
 -                    Row row;
 -                    try
 -                    {
 -                        row = handler.get();
 -                    }
 -                    catch (DigestMismatchException e)
 -                    {
 -                        throw new AssertionError(e); // full data requested from each node here, no digests should be sent
 -                    }
 -                    catch (ReadTimeoutException e)
 -                    {
 -                        if (Tracing.isTracing())
 -                            Tracing.trace("Timed out waiting on digest mismatch repair requests");
 -                        else
 -                            logger.trace("Timed out waiting on digest mismatch repair requests");
 -                        // the caught exception here will have CL.ALL from the repair command,
 -                        // not whatever CL the initial command was at (CASSANDRA-7947)
 -                        int blockFor = consistencyLevel.blockFor(Keyspace.open(command.getKeyspace()));
 -                        throw new ReadTimeoutException(consistencyLevel, blockFor-1, blockFor, true);
 -                    }
 +        void awaitResultsAndRetryOnDigestMismatch() throws ReadFailureException, ReadTimeoutException
 +        {
 +            try
 +            {
 +                result = executor.get();
 +            }
 +            catch (DigestMismatchException ex)
 +            {
 +                Tracing.trace("Digest mismatch: {}", ex);
  
 -                    RowDataResolver resolver = (RowDataResolver)handler.resolver;
 -                    try
 -                    {
 -                        // wait for the repair writes to be acknowledged, to minimize impact on any replica that's
 -                        // behind on writes in case the out-of-sync row is read multiple times in quick succession
 -                        FBUtilities.waitOnFutures(resolver.repairResults, DatabaseDescriptor.getWriteRpcTimeout());
 -                    }
 -                    catch (TimeoutException e)
 -                    {
 -                        if (Tracing.isTracing())
 -                            Tracing.trace("Timed out waiting on digest mismatch repair acknowledgements");
 -                        else
 -                            logger.trace("Timed out waiting on digest mismatch repair acknowledgements");
 -                        int blockFor = consistencyLevel.blockFor(Keyspace.open(command.getKeyspace()));
 -                        throw new ReadTimeoutException(consistencyLevel, blockFor-1, blockFor, true);
 -                    }
 +                ReadRepairMetrics.repairedBlocking.mark();
  
 -                    // retry any potential short reads
 -                    ReadCommand retryCommand = command.maybeGenerateRetryCommand(resolver, row);
 -                    if (retryCommand != null)
 -                    {
 -                        Tracing.trace("Issuing retry for read command");
 -                        if (commandsToRetry == Collections.EMPTY_LIST)
 -                            commandsToRetry = new ArrayList<>();
 -                        commandsToRetry.add(retryCommand);
 -                        continue;
 -                    }
 +                // Do a full data read to resolve the correct response (and repair node that need be)
 +                Keyspace keyspace = Keyspace.open(command.metadata().ksName);
 +                DataResolver resolver = new DataResolver(keyspace, command, ConsistencyLevel.ALL, executor.handler.endpoints.size());
 +                repairHandler = new ReadCallback(resolver,
 +                                                 ConsistencyLevel.ALL,
 +                                                 executor.getContactedReplicas().size(),
 +                                                 command,
 +                                                 keyspace,
 +                                                 executor.handler.endpoints);
  
 -                    if (row != null)
 -                    {
 -                        row = command.maybeTrim(row);
 -                        rows.add(row);
 -                    }
 +                for (InetAddress endpoint : executor.getContactedReplicas())
 +                {
 +                    MessageOut<ReadCommand> message = command.createMessage(MessagingService.instance().getVersion(endpoint));
 +                    Tracing.trace("Enqueuing full data read to {}", endpoint);
 +                    MessagingService.instance().sendRRWithFailure(message, endpoint, repairHandler);
                  }
              }
 -        } while (!commandsToRetry.isEmpty());
 -
 -        return rows;
 -    }
 -
 -    static class LocalReadRunnable extends DroppableRunnable
 -    {
 -        private final ReadCommand command;
 -        private final ReadCallback<ReadResponse, Row> handler;
 -        private final long start = System.nanoTime();
 -
 -        LocalReadRunnable(ReadCommand command, ReadCallback<ReadResponse, Row> handler)
 -        {
 -            super(MessagingService.Verb.READ);
 -            this.command = command;
 -            this.handler = handler;
          }
  
 -        protected void runMayThrow()
 +        void maybeAwaitFullDataRead() throws ReadTimeoutException
          {
 +            // There wasn't a digest mismatch, we're good
 +            if (repairHandler == null)
 +                return;
 +
 +            // Otherwise, get the result from the full-data read and check that it's not a short read
              try
              {
 -                Keyspace keyspace = Keyspace.open(command.ksName);
 -                Row r = command.getRow(keyspace);
 -                ReadResponse result = ReadVerbHandler.getResponse(command, r);
 -                MessagingService.instance().addLatency(FBUtilities.getBroadcastAddress(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 -                handler.response(result);
 +                result = repairHandler.get();
              }
 -            catch (Throwable t)
 +            catch (DigestMismatchException e)
              {
 -                handler.onFailure(FBUtilities.getBroadcastAddress());
 -                if (t instanceof TombstoneOverwhelmingException)
 -                    logger.error(t.getMessage());
 +                throw new AssertionError(e); // full data requested from each node here, no digests should be sent
 +            }
 +            catch (ReadTimeoutException e)
 +            {
 +                if (Tracing.isTracing())
 +                    Tracing.trace("Timed out waiting on digest mismatch repair requests");
                  else
-                     logger.debug("Timed out waiting on digest mismatch repair requests");
 -                    throw t;
++                    logger.trace("Timed out waiting on digest mismatch repair requests");
 +                // the caught exception here will have CL.ALL from the repair command,
 +                // not whatever CL the initial command was at (CASSANDRA-7947)
 +                int blockFor = consistency.blockFor(Keyspace.open(command.metadata().ksName));
 +                throw new ReadTimeoutException(consistency, blockFor-1, blockFor, true);
              }
          }
 +
 +        PartitionIterator getResult()
 +        {
 +            assert result != null;
 +            return result;
 +        }
      }
  
 -    static class LocalRangeSliceRunnable extends DroppableRunnable
 +    static class LocalReadRunnable extends DroppableRunnable
      {
 -        private final AbstractRangeCommand command;
 -        private final ReadCallback<RangeSliceReply, Iterable<Row>> handler;
 +        private final ReadCommand command;
 +        private final ReadCallback handler;
          private final long start = System.nanoTime();
  
 -        LocalRangeSliceRunnable(AbstractRangeCommand command, ReadCallback<RangeSliceReply, Iterable<Row>> handler)
 +        LocalReadRunnable(ReadCommand command, ReadCallback handler)
          {
 -            super(MessagingService.Verb.RANGE_SLICE);
 +            super(MessagingService.Verb.READ);
              this.command = command;
              this.handler = handler;
          }
@@@ -1832,199 -1661,253 +1832,199 @@@
          }
      }
  
 -    public static List<Row> getRangeSlice(AbstractRangeCommand command, ConsistencyLevel consistency_level)
 -    throws UnavailableException, ReadFailureException, ReadTimeoutException
 +    private static class SingleRangeResponse extends AbstractIterator<RowIterator> implements PartitionIterator
      {
 -        Tracing.trace("Computing ranges to query");
 -        long startTime = System.nanoTime();
 +        private final ReadCallback handler;
 +        private PartitionIterator result;
  
 -        Keyspace keyspace = Keyspace.open(command.keyspace);
 -        List<Row> rows;
 -        // now scan until we have enough results
 -        try
 +        private SingleRangeResponse(ReadCallback handler)
          {
 -            int liveRowCount = 0;
 -            boolean countLiveRows = command.countCQL3Rows() || command.ignoredTombstonedPartitions();
 -            rows = new ArrayList<>();
 +            this.handler = handler;
 +        }
  
 -            // when dealing with LocalStrategy keyspaces, we can skip the range splitting and merging (which can be
 -            // expensive in clusters with vnodes)
 -            List<? extends AbstractBounds<RowPosition>> ranges;
 -            if (keyspace.getReplicationStrategy() instanceof LocalStrategy)
 -                ranges = command.keyRange.unwrap();
 -            else
 -                ranges = getRestrictedRanges(command.keyRange);
 +        private void waitForResponse() throws ReadTimeoutException
 +        {
 +            if (result != null)
 +                return;
  
 -            // determine the number of rows to be fetched and the concurrency factor
 -            int rowsToBeFetched = command.limit();
 -            int concurrencyFactor;
 -            if (command.requiresScanningAllRanges())
 +            try
              {
 -                // all nodes must be queried
 -                rowsToBeFetched *= ranges.size();
 -                concurrencyFactor = ranges.size();
 -                logger.debug("Requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 -                             command.limit(),
 -                             ranges.size(),
 -                             concurrencyFactor);
 -                Tracing.trace("Submitting range requests on {} ranges with a concurrency of {}",
 -                              ranges.size(), concurrencyFactor);
 +                result = handler.get();
              }
 -            else
 +            catch (DigestMismatchException e)
              {
 -                // our estimate of how many result rows there will be per-range
 -                float resultRowsPerRange = estimateResultRowsPerRange(command, keyspace);
 -                // underestimate how many rows we will get per-range in order to increase the likelihood that we'll
 -                // fetch enough rows in the first round
 -                resultRowsPerRange -= resultRowsPerRange * CONCURRENT_SUBREQUESTS_MARGIN;
 -                concurrencyFactor = resultRowsPerRange == 0.0
 -                                  ? 1
 -                                  : Math.max(1, Math.min(ranges.size(), (int) Math.ceil(command.limit() / resultRowsPerRange)));
 -
 -                logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 -                             resultRowsPerRange,
 -                             command.limit(),
 -                             ranges.size(),
 -                             concurrencyFactor);
 -                Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)",
 -                              ranges.size(),
 -                              concurrencyFactor,
 -                              resultRowsPerRange);
 +                throw new AssertionError(e); // no digests in range slices yet
              }
 +        }
 +
 +        protected RowIterator computeNext()
 +        {
 +            waitForResponse();
 +            return result.hasNext() ? result.next() : endOfData();
 +        }
 +
 +        public void close()
 +        {
 +            if (result != null)
 +                result.close();
 +        }
 +    }
 +
 +    private static class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
 +    {
 +        private final Iterator<RangeForQuery> ranges;
 +        private final int totalRangeCount;
 +        private final PartitionRangeReadCommand command;
 +        private final Keyspace keyspace;
 +        private final ConsistencyLevel consistency;
 +
 +        private final long startTime;
 +        private CountingPartitionIterator sentQueryIterator;
 +
 +        private int concurrencyFactor;
 +        // The two following "metric" are maintained to improve the concurrencyFactor
 +        // when it was not good enough initially.
 +        private int liveReturned;
 +        private int rangesQueried;
 +
 +        public RangeCommandIterator(RangeIterator ranges, PartitionRangeReadCommand command, int concurrencyFactor, Keyspace keyspace, ConsistencyLevel consistency)
 +        {
 +            this.command = command;
 +            this.concurrencyFactor = concurrencyFactor;
 +            this.startTime = System.nanoTime();
 +            this.ranges = new RangeMerger(ranges, keyspace, consistency);
 +            this.totalRangeCount = ranges.rangeCount();
 +            this.consistency = consistency;
 +            this.keyspace = keyspace;
 +        }
  
 -            boolean haveSufficientRows = false;
 -            int i = 0;
 -            AbstractBounds<RowPosition> nextRange = null;
 -            List<InetAddress> nextEndpoints = null;
 -            List<InetAddress> nextFilteredEndpoints = null;
 -            while (i < ranges.size())
 +        public RowIterator computeNext()
 +        {
 +            while (sentQueryIterator == null || !sentQueryIterator.hasNext())
              {
 -                List<Pair<AbstractRangeCommand, ReadCallback<RangeSliceReply, Iterable<Row>>>> scanHandlers = new ArrayList<>(concurrencyFactor);
 -                int concurrentFetchStartingIndex = i;
 -                int concurrentRequests = 0;
 -                while ((i - concurrentFetchStartingIndex) < concurrencyFactor)
 +                // If we don't have more range to handle, we're done
 +                if (!ranges.hasNext())
 +                    return endOfData();
 +
 +                // else, sends the next batch of concurrent queries (after having close the previous iterator)
 +                if (sentQueryIterator != null)
                  {
 -                    AbstractBounds<RowPosition> range = nextRange == null
 -                                                      ? ranges.get(i)
 -                                                      : nextRange;
 -                    List<InetAddress> liveEndpoints = nextEndpoints == null
 -                                                    ? getLiveSortedEndpoints(keyspace, range.right)
 -                                                    : nextEndpoints;
 -                    List<InetAddress> filteredEndpoints = nextFilteredEndpoints == null
 -                                                        ? consistency_level.filterForQuery(keyspace, liveEndpoints)
 -                                                        : nextFilteredEndpoints;
 -                    ++i;
 -                    ++concurrentRequests;
 -
 -                    // getRestrictedRange has broken the queried range into per-[vnode] token ranges, but this doesn't take
 -                    // the replication factor into account. If the intersection of live endpoints for 2 consecutive ranges
 -                    // still meets the CL requirements, then we can merge both ranges into the same RangeSliceCommand.
 -                    while (i < ranges.size())
 -                    {
 -                        nextRange = ranges.get(i);
 -                        nextEndpoints = getLiveSortedEndpoints(keyspace, nextRange.right);
 -                        nextFilteredEndpoints = consistency_level.filterForQuery(keyspace, nextEndpoints);
 -
 -                        // If the current range right is the min token, we should stop merging because CFS.getRangeSlice
 -                        // don't know how to deal with a wrapping range.
 -                        // Note: it would be slightly more efficient to have CFS.getRangeSlice on the destination nodes unwraps
 -                        // the range if necessary and deal with it. However, we can't start sending wrapped range without breaking
 -                        // wire compatibility, so It's likely easier not to bother;
 -                        if (range.right.isMinimum())
 -                            break;
 -
 -                        List<InetAddress> merged = intersection(liveEndpoints, nextEndpoints);
 -
 -                        // Check if there is enough endpoint for the merge to be possible.
 -                        if (!consistency_level.isSufficientLiveNodes(keyspace, merged))
 -                            break;
 -
 -                        List<InetAddress> filteredMerged = consistency_level.filterForQuery(keyspace, merged);
 -
 -                        // Estimate whether merging will be a win or not
 -                        if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(filteredMerged, filteredEndpoints, nextFilteredEndpoints))
 -                            break;
 -
 -                        // If we get there, merge this range and the next one
 -                        range = range.withNewRight(nextRange.right);
 -                        liveEndpoints = merged;
 -                        filteredEndpoints = filteredMerged;
 -                        ++i;
 -                    }
 +                    liveReturned += sentQueryIterator.counter().counted();
 +                    sentQueryIterator.close();
  
 -                    AbstractRangeCommand nodeCmd = command.forSubRange(range);
 -
 -                    // collect replies and resolve according to consistency level
 -                    RangeSliceResponseResolver resolver = new RangeSliceResponseResolver(nodeCmd.keyspace, command.timestamp);
 -                    List<InetAddress> minimalEndpoints = filteredEndpoints.subList(0, Math.min(filteredEndpoints.size(), consistency_level.blockFor(keyspace)));
 -                    ReadCallback<RangeSliceReply, Iterable<Row>> handler = new ReadCallback<>(resolver, consistency_level, nodeCmd, minimalEndpoints);
 -                    handler.assureSufficientLiveNodes();
 -                    resolver.setSources(filteredEndpoints);
 -                    if (filteredEndpoints.size() == 1
 -                        && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress())
 -                        && OPTIMIZE_LOCAL_REQUESTS)
 -                    {
 -                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler), Tracing.instance.get());
 -                    }
 -                    else
 -                    {
 -                        MessageOut<? extends AbstractRangeCommand> message = nodeCmd.createMessage();
 -                        for (InetAddress endpoint : filteredEndpoints)
 -                        {
 -                            Tracing.trace("Enqueuing request to {}", endpoint);
 -                            MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
 -                        }
 -                    }
 -                    scanHandlers.add(Pair.create(nodeCmd, handler));
 +                    // It's not the first batch of queries and we're not done, so we we can use what has been
 +                    // returned so far to improve our rows-per-range estimate and update the concurrency accordingly
 +                    updateConcurrencyFactor();
                  }
 -                Tracing.trace("Submitted {} concurrent range requests covering {} ranges", concurrentRequests, i - concurrentFetchStartingIndex);
 +                sentQueryIterator = sendNextRequests();
 +            }
  
 -                List<AsyncOneResponse> repairResponses = new ArrayList<>();
 -                for (Pair<AbstractRangeCommand, ReadCallback<RangeSliceReply, Iterable<Row>>> cmdPairHandler : scanHandlers)
 -                {
 -                    ReadCallback<RangeSliceReply, Iterable<Row>> handler = cmdPairHandler.right;
 -                    RangeSliceResponseResolver resolver = (RangeSliceResponseResolver)handler.resolver;
 +            return sentQueryIterator.next();
 +        }
  
 -                    try
 -                    {
 -                        for (Row row : handler.get())
 -                        {
 -                            rows.add(row);
 -                            if (countLiveRows)
 -                                liveRowCount += row.getLiveCount(command.predicate, command.timestamp);
 -                        }
 -                        repairResponses.addAll(resolver.repairResults);
 -                    }
 -                    catch (ReadTimeoutException|ReadFailureException ex)
 -                    {
 -                        // we timed out or failed waiting for responses
 -                        int blockFor = consistency_level.blockFor(keyspace);
 -                        int responseCount = resolver.responses.size();
 -                        String gotData = responseCount > 0
 -                                         ? resolver.isDataPresent() ? " (including data)" : " (only digests)"
 -                                         : "";
 -
 -                        boolean isTimeout = ex instanceof ReadTimeoutException;
 -                        if (Tracing.isTracing())
 -                        {
 -                            Tracing.trace("{}; received {} of {} responses{} for range {} of {}",
 -                                          (isTimeout ? "Timed out" : "Failed"), responseCount, blockFor, gotData, i, ranges.size());
 -                        }
 -                        else if (logger.isDebugEnabled())
 -                        {
 -                            logger.debug("Range slice {}; received {} of {} responses{} for range {} of {}",
 -                                         (isTimeout ? "timeout" : "failure"), responseCount, blockFor, gotData, i, ranges.size());
 -                        }
 -                        throw ex;
 -                    }
 -                    catch (DigestMismatchException e)
 -                    {
 -                        throw new AssertionError(e); // no digests in range slices yet
 -                    }
 +        private void updateConcurrencyFactor()
 +        {
 +            if (liveReturned == 0)
 +            {
 +                // we haven't actually gotten any results, so query all remaining ranges at once
 +                concurrencyFactor = totalRangeCount - rangesQueried;
 +                return;
 +            }
  
 -                    // if we're done, great, otherwise, move to the next range
 -                    int count = countLiveRows ? liveRowCount : rows.size();
 -                    if (count >= rowsToBeFetched)
 -                    {
 -                        haveSufficientRows = true;
 -                        break;
 -                    }
 -                }
 +            // Otherwise, compute how many rows per range we got on average and pick a concurrency factor
 +            // that should allow us to fetch all remaining rows with the next batch of (concurrent) queries.
 +            int remainingRows = command.limits().count() - liveReturned;
 +            float rowsPerRange = (float)liveReturned / (float)rangesQueried;
 +            concurrencyFactor = Math.max(1, Math.min(totalRangeCount - rangesQueried, Math.round(remainingRows / rowsPerRange)));
-             logger.debug("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
++            logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
 +                         rowsPerRange, (int) remainingRows, concurrencyFactor);
 +        }
  
 -                try
 -                {
 -                    FBUtilities.waitOnFutures(repairResponses, DatabaseDescriptor.getWriteRpcTimeout());
 -                }
 -                catch (TimeoutException ex)
 -                {
 -                    // We got all responses, but timed out while repairing
 -                    int blockFor = consistency_level.blockFor(keyspace);
 -                    if (Tracing.isTracing())
 -                        Tracing.trace("Timed out while read-repairing after receiving all {} data and digest responses", blockFor);
 -                    else
 -                        logger.debug("Range slice timeout while read-repairing after receiving all {} data and digest responses", blockFor);
 -                    throw new ReadTimeoutException(consistency_level, blockFor-1, blockFor, true);
 -                }
 +        private SingleRangeResponse query(RangeForQuery toQuery)
 +        {
 +            PartitionRangeReadCommand rangeCommand = command.forSubRange(toQuery.range);
 +
 +            DataResolver resolver = new DataResolver(keyspace, rangeCommand, consistency, toQuery.filteredEndpoints.size());
 +
 +            int blockFor = consistency.blockFor(keyspace);
 +            int minResponses = Math.min(toQuery.filteredEndpoints.size(), blockFor);
 +            List<InetAddress> minimalEndpoints = toQuery.filteredEndpoints.subList(0, minResponses);
 +            ReadCallback handler = new ReadCallback(resolver, consistency, rangeCommand, minimalEndpoints);
  
 -                if (haveSufficientRows)
 -                    return command.postReconciliationProcessing(rows);
 +            handler.assureSufficientLiveNodes();
  
 -                // we didn't get enough rows in our concurrent fetch; recalculate our concurrency factor
 -                // based on the results we've seen so far (as long as we still have ranges left to query)
 -                if (i < ranges.size())
 +            if (toQuery.filteredEndpoints.size() == 1 && canDoLocalRequest(toQuery.filteredEndpoints.get(0)))
 +            {
 +                StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(rangeCommand, handler), Tracing.instance.get());
 +            }
 +            else
 +            {
 +                for (InetAddress endpoint : toQuery.filteredEndpoints)
                  {
 -                    float fetchedRows = countLiveRows ? liveRowCount : rows.size();
 -                    float remainingRows = rowsToBeFetched - fetchedRows;
 -                    float actualRowsPerRange;
 -                    if (fetchedRows == 0.0)
 -                    {
 -                        // we haven't actually gotten any results, so query all remaining ranges at once
 -                        actualRowsPerRange = 0.0f;
 -                        concurrencyFactor = ranges.size() - i;
 -                    }
 -                    else
 -                    {
 -                        actualRowsPerRange = fetchedRows / i;
 -                        concurrencyFactor = Math.max(1, Math.min(ranges.size() - i, Math.round(remainingRows / actualRowsPerRange)));
 -                    }
 -                    logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
 -                                 actualRowsPerRange, (int) remainingRows, concurrencyFactor);
 +                    MessageOut<ReadCommand> message = rangeCommand.createMessage(MessagingService.instance().getVersion(endpoint));
 +                    Tracing.trace("Enqueuing request to {}", endpoint);
 +                    MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
                  }
              }
 +
 +            return new SingleRangeResponse(handler);
          }
 -        finally
 +
 +        private CountingPartitionIterator sendNextRequests()
 +        {
 +            List<PartitionIterator> concurrentQueries = new ArrayList<>(concurrencyFactor);
 +            for (int i = 0; i < concurrencyFactor && ranges.hasNext(); i++)
 +            {
 +                concurrentQueries.add(query(ranges.next()));
 +                ++rangesQueried;
 +            }
 +
 +            Tracing.trace("Submitted {} concurrent range requests", concurrentQueries.size());
 +            // We want to count the results for the sake of updating the concurrency factor (see updateConcurrencyFactor) but we don't want to
 +            // enforce any particular limit at this point (this could break code than rely on postReconciliationProcessing), hence the DataLimits.NONE.
 +            return new CountingPartitionIterator(PartitionIterators.concat(concurrentQueries), DataLimits.NONE, command.nowInSec());
 +        }
 +
 +        public void close()
          {
 -            long latency = System.nanoTime() - startTime;
 -            rangeMetrics.addNano(latency);
 -            Keyspace.open(command.keyspace).getColumnFamilyStore(command.columnFamily).metric.coordinatorScanLatency.update(latency, TimeUnit.NANOSECONDS);
 +            try
 +            {
 +                if (sentQueryIterator != null)
 +                    sentQueryIterator.close();
 +            }
 +            finally
 +            {
 +                long latency = System.nanoTime() - startTime;
 +                rangeMetrics.addNano(latency);
 +                Keyspace.openAndGetStore(command.metadata()).metric.coordinatorScanLatency.update(latency, TimeUnit.NANOSECONDS);
 +            }
          }
 -        return command.postReconciliationProcessing(rows);
 +    }
 +
 +    @SuppressWarnings("resource")
 +    public static PartitionIterator getRangeSlice(PartitionRangeReadCommand command, ConsistencyLevel consistencyLevel)
 +    throws UnavailableException, ReadFailureException, ReadTimeoutException
 +    {
 +        Tracing.trace("Computing ranges to query");
 +
 +        Keyspace keyspace = Keyspace.open(command.metadata().ksName);
 +        RangeIterator ranges = new RangeIterator(command, keyspace, consistencyLevel);
 +
 +        // our estimate of how many result rows there will be per-range
 +        float resultsPerRange = estimateResultsPerRange(command, keyspace);
 +        // underestimate how many rows we will get per-range in order to increase the likelihood that we'll
 +        // fetch enough rows in the first round
 +        resultsPerRange -= resultsPerRange * CONCURRENT_SUBREQUESTS_MARGIN;
 +        int concurrencyFactor = resultsPerRange == 0.0
 +                              ? 1
 +                              : Math.max(1, Math.min(ranges.rangeCount(), (int) Math.ceil(command.limits().count() / resultsPerRange)));
-         logger.debug("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
++        logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 +                     resultsPerRange, command.limits().count(), ranges.rangeCount(), concurrencyFactor);
 +        Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)", ranges.rangeCount(), concurrencyFactor, resultsPerRange);
 +
 +        // Note that in general, a RangeCommandIterator will honor the command limit for each range, but will not enforce it globally.
 +
 +        return command.limits().filter(command.postReconciliationProcessing(new RangeCommandIterator(ranges, command, concurrencyFactor, keyspace, consistencyLevel)), command.nowInSec());
      }
  
      public Map<String, List<String>> getSchemaVersions()
@@@ -2430,52 -2287,6 +2430,52 @@@
              logger.warn("Some hints were not written before shutdown.  This is not supposed to happen.  You should (a) run repair, and (b) file a bug report");
      }
  
 +    private static AtomicInteger getHintsInProgressFor(InetAddress destination)
 +    {
 +        try
 +        {
 +            return hintsInProgress.load(destination);
 +        }
 +        catch (Exception e)
 +        {
 +            throw new AssertionError(e);
 +        }
 +    }
 +
 +    public static Future<Void> submitHint(Mutation mutation, InetAddress target, AbstractWriteResponseHandler<IMutation> responseHandler)
 +    {
 +        return submitHint(mutation, Collections.singleton(target), responseHandler);
 +    }
 +
 +    public static Future<Void> submitHint(Mutation mutation,
 +                                          Collection<InetAddress> targets,
 +                                          AbstractWriteResponseHandler<IMutation> responseHandler)
 +    {
 +        HintRunnable runnable = new HintRunnable(targets)
 +        {
 +            public void runMayThrow()
 +            {
-                 logger.debug("Adding hints for {}", targets);
++                logger.trace("Adding hints for {}", targets);
 +                HintsService.instance.write(Iterables.transform(targets, StorageService.instance::getHostIdForEndpoint),
 +                                            Hint.create(mutation, System.currentTimeMillis()));
 +                targets.forEach(HintsService.instance.metrics::incrCreatedHints);
 +                // Notify the handler only for CL == ANY
 +                if (responseHandler != null && responseHandler.consistencyLevel == ConsistencyLevel.ANY)
 +                    responseHandler.response(null);
 +            }
 +        };
 +
 +        return submitHint(runnable);
 +    }
 +
 +    private static Future<Void> submitHint(HintRunnable runnable)
 +    {
 +        StorageMetrics.totalHintsInProgress.inc(runnable.targets.size());
 +        for (InetAddress target : runnable.targets)
 +            getHintsInProgressFor(target).incrementAndGet();
 +        return (Future<Void>) StageManager.getStage(Stage.MUTATION).submit(runnable);
 +    }
 +
      public Long getRpcTimeout() { return DatabaseDescriptor.getRpcTimeout(); }
      public void setRpcTimeout(Long timeoutInMillis) { DatabaseDescriptor.setRpcTimeout(timeoutInMillis); }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CassandraServer.java
index 7017bc1,1abc928..538d128
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@@ -688,10 -538,10 +688,10 @@@ public class CassandraServer implement
              // request by page if this is a large row
              if (cfs.getMeanColumns() > 0)
              {
 -                int averageColumnSize = (int) (cfs.metric.meanRowSize.getValue() / cfs.getMeanColumns());
 +                int averageColumnSize = (int) (cfs.metric.meanPartitionSize.getValue() / cfs.getMeanColumns());
                  pageSize = Math.min(COUNT_PAGE_SIZE, 4 * 1024 * 1024 / averageColumnSize);
                  pageSize = Math.max(2, pageSize);
-                 logger.debug("average row column size is {}; using pageSize of {}", averageColumnSize, pageSize);
+                 logger.trace("average row column size is {}; using pageSize of {}", averageColumnSize, pageSize);
              }
              else
              {
@@@ -1866,9 -1492,10 +1866,9 @@@
          requestScheduler.release();
      }
  
 -    public String system_add_column_family(CfDef cf_def)
 -    throws InvalidRequestException, SchemaDisagreementException, TException
 +    public String system_add_column_family(CfDef cf_def) throws TException
      {
-         logger.debug("add_column_family");
+         logger.trace("add_column_family");
  
          try
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/utils/CLibrary.java
----------------------------------------------------------------------


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

Posted by bl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index c4ef239,87891ae..8d23597
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@@ -409,14 -399,9 +409,14 @@@ public abstract class SSTableReader ext
              System.exit(1);
          }
  
-         logger.info("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
+         logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
 -        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
 -                statsMetadata, OpenReason.NORMAL);
 +        SSTableReader sstable = internalOpen(descriptor,
 +                                             components,
 +                                             metadata,
 +                                             System.currentTimeMillis(),
 +                                             statsMetadata,
 +                                             OpenReason.NORMAL,
 +                                             header.toHeader(metadata));
  
          // special implementation of load to use non-pooled SegmentedFile builders
          try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();
@@@ -465,15 -446,9 +465,15 @@@
              System.exit(1);
          }
  
-         logger.info("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
+         logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
 -        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
 -                                             statsMetadata, OpenReason.NORMAL);
 +        SSTableReader sstable = internalOpen(descriptor,
 +                                             components,
 +                                             metadata,
 +                                             System.currentTimeMillis(),
 +                                             statsMetadata,
 +                                             OpenReason.NORMAL,
 +                                             header == null ? null : header.toHeader(metadata));
 +
          try
          {
              // load index and filter
@@@ -1656,10 -1631,10 +1656,10 @@@
       * @return true if the this is the first time the file was marked obsolete.  Calling this
       * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
       */
 -    public boolean markObsolete(Tracker tracker)
 +    public void markObsolete(Runnable tidier)
      {
-         if (logger.isDebugEnabled())
-             logger.debug("Marking {} compacted", getFilename());
+         if (logger.isTraceEnabled())
+             logger.trace("Marking {} compacted", getFilename());
  
          synchronized (tidy.global)
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
index 9197b7a,30ed85b..635adcd
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
@@@ -76,10 -75,10 +76,10 @@@ public class MetadataSerializer impleme
          }
      }
  
 -    public Map<MetadataType, MetadataComponent> deserialize(Descriptor descriptor, EnumSet<MetadataType> types) throws IOException
 +    public Map<MetadataType, MetadataComponent> deserialize( Descriptor descriptor, EnumSet<MetadataType> types) throws IOException
      {
          Map<MetadataType, MetadataComponent> components;
-         logger.debug("Load metadata for {}", descriptor);
+         logger.trace("Load metadata for {}", descriptor);
          File statsFile = new File(descriptor.filenameFor(Component.STATS));
          if (!statsFile.exists())
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/locator/TokenMetadata.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/net/IncomingTcpConnection.java
index 8e8ce15,f6652b0..7054bcc
--- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
@@@ -149,10 -148,10 +149,10 @@@ public class IncomingTcpConnection exte
  
          if (compressed)
          {
-             logger.debug("Upgrading incoming connection to be compressed");
+             logger.trace("Upgrading incoming connection to be compressed");
              if (version < MessagingService.VERSION_21)
              {
 -                in = new DataInputStream(new SnappyInputStream(socket.getInputStream()));
 +                in = new DataInputStreamPlus(new SnappyInputStream(socket.getInputStream()));
              }
              else
              {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/net/MessagingService.java
index 4fb67ec,2a63553..810d086
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@@ -874,24 -847,17 +874,24 @@@ public final class MessagingService imp
  
      public void resetVersion(InetAddress endpoint)
      {
-         logger.debug("Resetting version for {}", endpoint);
+         logger.trace("Resetting version for {}", endpoint);
          Integer removed = versions.remove(endpoint);
 -        if (removed != null && removed <= VERSION_22)
 -            refreshAllNodesAtLeast22();
 +        if (removed != null && removed <= VERSION_30)
 +            refreshAllNodeMinVersions();
      }
  
 -    private void refreshAllNodesAtLeast22()
 +    private void refreshAllNodeMinVersions()
      {
 -        for (Integer version: versions.values())
 +        boolean anyNodeLowerThan30 = false;
 +        for (Integer version : versions.values())
          {
 -            if (version < VERSION_22)
 +            if (version < MessagingService.VERSION_30)
 +            {
 +                anyNodeLowerThan30 = true;
 +                allNodesAtLeast30 = false;
 +            }
 +
 +            if (version < MessagingService.VERSION_22)
              {
                  allNodesAtLeast22 = false;
                  return;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index 5f27d82,0000000..bc9da31
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@@ -1,1708 -1,0 +1,1708 @@@
 +/*
 + * 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.schema;
 +
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.nio.charset.CharacterCodingException;
 +import java.security.MessageDigest;
 +import java.security.NoSuchAlgorithmException;
 +import java.util.*;
 +import java.util.concurrent.TimeUnit;
 +import java.util.function.Function;
 +import java.util.stream.Collectors;
 +
 +import com.google.common.collect.ImmutableList;
 +import com.google.common.collect.MapDifference;
 +import com.google.common.collect.Maps;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.config.*;
 +import org.apache.cassandra.config.ColumnDefinition.ClusteringOrder;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.functions.*;
 +import org.apache.cassandra.cql3.statements.SelectStatement;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.db.partitions.*;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.db.view.View;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +
 +import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
 +
 +/**
 + * system_schema.* tables and methods for manipulating them.
 + */
 +public final class SchemaKeyspace
 +{
 +    private SchemaKeyspace()
 +    {
 +    }
 +
 +    private static final Logger logger = LoggerFactory.getLogger(SchemaKeyspace.class);
 +
 +    public static final String NAME = "system_schema";
 +
 +    public static final String KEYSPACES = "keyspaces";
 +    public static final String TABLES = "tables";
 +    public static final String COLUMNS = "columns";
 +    public static final String DROPPED_COLUMNS = "dropped_columns";
 +    public static final String TRIGGERS = "triggers";
 +    public static final String VIEWS = "views";
 +    public static final String TYPES = "types";
 +    public static final String FUNCTIONS = "functions";
 +    public static final String AGGREGATES = "aggregates";
 +    public static final String INDEXES = "indexes";
 +
 +
 +    public static final List<String> ALL =
 +        ImmutableList.of(KEYSPACES, TABLES, COLUMNS, TRIGGERS, VIEWS, TYPES, FUNCTIONS, AGGREGATES, INDEXES);
 +
 +    private static final CFMetaData Keyspaces =
 +        compile(KEYSPACES,
 +                "keyspace definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "durable_writes boolean,"
 +                + "replication frozen<map<text, text>>,"
 +                + "PRIMARY KEY ((keyspace_name)))");
 +
 +    private static final CFMetaData Tables =
 +        compile(TABLES,
 +                "table definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "table_name text,"
 +                + "bloom_filter_fp_chance double,"
 +                + "caching frozen<map<text, text>>,"
 +                + "comment text,"
 +                + "compaction frozen<map<text, text>>,"
 +                + "compression frozen<map<text, text>>,"
 +                + "crc_check_chance double,"
 +                + "dclocal_read_repair_chance double,"
 +                + "default_time_to_live int,"
 +                + "extensions frozen<map<text, blob>>,"
 +                + "flags frozen<set<text>>," // SUPER, COUNTER, DENSE, COMPOUND
 +                + "gc_grace_seconds int,"
 +                + "id uuid,"
 +                + "max_index_interval int,"
 +                + "memtable_flush_period_in_ms int,"
 +                + "min_index_interval int,"
 +                + "read_repair_chance double,"
 +                + "speculative_retry text,"
 +                + "PRIMARY KEY ((keyspace_name), table_name))");
 +
 +    private static final CFMetaData Columns =
 +        compile(COLUMNS,
 +                "column definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "table_name text,"
 +                + "column_name text,"
 +                + "clustering_order text,"
 +                + "column_name_bytes blob,"
 +                + "kind text,"
 +                + "position int,"
 +                + "type text,"
 +                + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
 +
 +    private static final CFMetaData DroppedColumns =
 +        compile(DROPPED_COLUMNS,
 +                "dropped column registry",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "table_name text,"
 +                + "column_name text,"
 +                + "dropped_time timestamp,"
 +                + "type text,"
 +                + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
 +
 +    private static final CFMetaData Triggers =
 +        compile(TRIGGERS,
 +                "trigger definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "table_name text,"
 +                + "trigger_name text,"
 +                + "options frozen<map<text, text>>,"
 +                + "PRIMARY KEY ((keyspace_name), table_name, trigger_name))");
 +
 +    private static final CFMetaData Views =
 +        compile(VIEWS,
 +                "view definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "view_name text,"
 +                + "base_table_id uuid,"
 +                + "base_table_name text,"
 +                + "where_clause text,"
 +                + "bloom_filter_fp_chance double,"
 +                + "caching frozen<map<text, text>>,"
 +                + "comment text,"
 +                + "compaction frozen<map<text, text>>,"
 +                + "compression frozen<map<text, text>>,"
 +                + "crc_check_chance double,"
 +                + "dclocal_read_repair_chance double,"
 +                + "default_time_to_live int,"
 +                + "extensions frozen<map<text, blob>>,"
 +                + "gc_grace_seconds int,"
 +                + "id uuid,"
 +                + "include_all_columns boolean,"
 +                + "max_index_interval int,"
 +                + "memtable_flush_period_in_ms int,"
 +                + "min_index_interval int,"
 +                + "read_repair_chance double,"
 +                + "speculative_retry text,"
 +                + "PRIMARY KEY ((keyspace_name), view_name))");
 +
 +    private static final CFMetaData Indexes =
 +        compile(INDEXES,
 +                "secondary index definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "table_name text,"
 +                + "index_name text,"
 +                + "kind text,"
 +                + "options frozen<map<text, text>>,"
 +                + "PRIMARY KEY ((keyspace_name), table_name, index_name))");
 +
 +    private static final CFMetaData Types =
 +        compile(TYPES,
 +                "user defined type definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "type_name text,"
 +                + "field_names frozen<list<text>>,"
 +                + "field_types frozen<list<text>>,"
 +                + "PRIMARY KEY ((keyspace_name), type_name))");
 +
 +    private static final CFMetaData Functions =
 +        compile(FUNCTIONS,
 +                "user defined function definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "function_name text,"
 +                + "signature frozen<list<text>>,"
 +                + "argument_names frozen<list<text>>,"
 +                + "argument_types frozen<list<text>>,"
 +                + "body text,"
 +                + "language text,"
 +                + "return_type text,"
 +                + "called_on_null_input boolean,"
 +                + "PRIMARY KEY ((keyspace_name), function_name, signature))");
 +
 +    private static final CFMetaData Aggregates =
 +        compile(AGGREGATES,
 +                "user defined aggregate definitions",
 +                "CREATE TABLE %s ("
 +                + "keyspace_name text,"
 +                + "aggregate_name text,"
 +                + "signature frozen<list<text>>,"
 +                + "argument_types frozen<list<text>>,"
 +                + "final_func text,"
 +                + "initcond blob,"
 +                + "return_type text,"
 +                + "state_func text,"
 +                + "state_type text,"
 +                + "PRIMARY KEY ((keyspace_name), aggregate_name, signature))");
 +
 +    public static final List<CFMetaData> ALL_TABLE_METADATA =
 +        ImmutableList.of(Keyspaces, Tables, Columns, Triggers, DroppedColumns, Views, Types, Functions, Aggregates, Indexes);
 +
 +    private static CFMetaData compile(String name, String description, String schema)
 +    {
 +        return CFMetaData.compile(String.format(schema, name), NAME)
 +                         .comment(description)
 +                         .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(7));
 +    }
 +
 +    public static KeyspaceMetadata metadata()
 +    {
 +        return KeyspaceMetadata.create(NAME, KeyspaceParams.local(), org.apache.cassandra.schema.Tables.of(ALL_TABLE_METADATA));
 +    }
 +
 +    /**
 +     * Add entries to system_schema.* for the hardcoded system keyspaces
 +     */
 +    public static void saveSystemKeyspacesSchema()
 +    {
 +        KeyspaceMetadata system = Schema.instance.getKSMetaData(SystemKeyspace.NAME);
 +        KeyspaceMetadata schema = Schema.instance.getKSMetaData(NAME);
 +
 +        long timestamp = FBUtilities.timestampMicros();
 +
 +        // delete old, possibly obsolete entries in schema tables
 +        for (String schemaTable : ALL)
 +        {
 +            String query = String.format("DELETE FROM %s.%s USING TIMESTAMP ? WHERE keyspace_name = ?", NAME, schemaTable);
 +            for (String systemKeyspace : Schema.SYSTEM_KEYSPACE_NAMES)
 +                executeOnceInternal(query, timestamp, systemKeyspace);
 +        }
 +
 +        // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
 +        makeCreateKeyspaceMutation(system, timestamp + 1).apply();
 +        makeCreateKeyspaceMutation(schema, timestamp + 1).apply();
 +    }
 +
 +    public static List<KeyspaceMetadata> readSchemaFromSystemTables()
 +    {
 +        ReadCommand cmd = getReadCommandForTableSchema(KEYSPACES);
 +        try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); PartitionIterator schema = cmd.executeInternal(orderGroup))
 +        {
 +            List<KeyspaceMetadata> keyspaces = new ArrayList<>();
 +
 +            while (schema.hasNext())
 +            {
 +                try (RowIterator partition = schema.next())
 +                {
 +                    if (isSystemKeyspaceSchemaPartition(partition.partitionKey()))
 +                        continue;
 +
 +                    DecoratedKey key = partition.partitionKey();
 +
 +                    readSchemaPartitionForKeyspaceAndApply(TYPES, key,
 +                        types -> readSchemaPartitionForKeyspaceAndApply(TABLES, key,
 +                        tables -> readSchemaPartitionForKeyspaceAndApply(VIEWS, key,
 +                        views -> readSchemaPartitionForKeyspaceAndApply(FUNCTIONS, key,
 +                        functions -> readSchemaPartitionForKeyspaceAndApply(AGGREGATES, key,
 +                        aggregates -> keyspaces.add(createKeyspaceFromSchemaPartitions(partition, tables, views, types, functions, aggregates))))))
 +                    );
 +                }
 +            }
 +            return keyspaces;
 +        }
 +    }
 +
 +    public static void truncate()
 +    {
 +        ALL.forEach(table -> getSchemaCFS(table).truncateBlocking());
 +    }
 +
 +    static void flush()
 +    {
 +        if (!Boolean.getBoolean("cassandra.unsafesystem"))
 +            ALL.forEach(table -> FBUtilities.waitOnFuture(getSchemaCFS(table).forceFlush()));
 +    }
 +
 +    /**
 +     * Read schema from system keyspace and calculate MD5 digest of every row, resulting digest
 +     * will be converted into UUID which would act as content-based version of the schema.
 +     */
 +    public static UUID calculateSchemaDigest()
 +    {
 +        MessageDigest digest;
 +        try
 +        {
 +            digest = MessageDigest.getInstance("MD5");
 +        }
 +        catch (NoSuchAlgorithmException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +
 +        for (String table : ALL)
 +        {
 +            ReadCommand cmd = getReadCommandForTableSchema(table);
 +            try (ReadOrderGroup orderGroup = cmd.startOrderGroup();
 +                 PartitionIterator schema = cmd.executeInternal(orderGroup))
 +            {
 +                while (schema.hasNext())
 +                {
 +                    try (RowIterator partition = schema.next())
 +                    {
 +                        if (!isSystemKeyspaceSchemaPartition(partition.partitionKey()))
 +                            RowIterators.digest(partition, digest);
 +                    }
 +                }
 +            }
 +        }
 +        return UUID.nameUUIDFromBytes(digest.digest());
 +    }
 +
 +    /**
 +     * @param schemaTableName The name of the table responsible for part of the schema
 +     * @return CFS responsible to hold low-level serialized schema
 +     */
 +    private static ColumnFamilyStore getSchemaCFS(String schemaTableName)
 +    {
 +        return Keyspace.open(NAME).getColumnFamilyStore(schemaTableName);
 +    }
 +
 +    /**
 +     * @param schemaTableName The name of the table responsible for part of the schema.
 +     * @return low-level schema representation
 +     */
 +    private static ReadCommand getReadCommandForTableSchema(String schemaTableName)
 +    {
 +        ColumnFamilyStore cfs = getSchemaCFS(schemaTableName);
 +        return PartitionRangeReadCommand.allDataRead(cfs.metadata, FBUtilities.nowInSeconds());
 +    }
 +
 +    public static Collection<Mutation> convertSchemaToMutations()
 +    {
 +        Map<DecoratedKey, Mutation> mutationMap = new HashMap<>();
 +
 +        for (String table : ALL)
 +            convertSchemaToMutations(mutationMap, table);
 +
 +        return mutationMap.values();
 +    }
 +
 +    private static void convertSchemaToMutations(Map<DecoratedKey, Mutation> mutationMap, String schemaTableName)
 +    {
 +        ReadCommand cmd = getReadCommandForTableSchema(schemaTableName);
 +        try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator iter = cmd.executeLocally(orderGroup))
 +        {
 +            while (iter.hasNext())
 +            {
 +                try (UnfilteredRowIterator partition = iter.next())
 +                {
 +                    if (isSystemKeyspaceSchemaPartition(partition.partitionKey()))
 +                        continue;
 +
 +                    DecoratedKey key = partition.partitionKey();
 +                    Mutation mutation = mutationMap.get(key);
 +                    if (mutation == null)
 +                    {
 +                        mutation = new Mutation(NAME, key);
 +                        mutationMap.put(key, mutation);
 +                    }
 +
 +                    mutation.add(PartitionUpdate.fromIterator(partition));
 +                }
 +            }
 +        }
 +    }
 +
 +    private static Map<DecoratedKey, FilteredPartition> readSchemaForKeyspaces(String schemaTableName, Set<String> keyspaceNames)
 +    {
 +        Map<DecoratedKey, FilteredPartition> schema = new HashMap<>();
 +
 +        for (String keyspaceName : keyspaceNames)
 +        {
 +            // We don't to return the RowIterator directly because we should guarantee that this iterator
 +            // will be closed, and putting it in a Map make that harder/more awkward.
 +            readSchemaPartitionForKeyspaceAndApply(schemaTableName, keyspaceName,
 +                partition -> {
 +                    if (!partition.isEmpty())
 +                        schema.put(partition.partitionKey(), FilteredPartition.create(partition));
 +                    return null;
 +                }
 +            );
 +        }
 +
 +        return schema;
 +    }
 +
 +    private static ByteBuffer getSchemaKSKey(String ksName)
 +    {
 +        return AsciiType.instance.fromString(ksName);
 +    }
 +
 +    private static <T> T readSchemaPartitionForKeyspaceAndApply(String schemaTableName, String keyspaceName, Function<RowIterator, T> fct)
 +    {
 +        return readSchemaPartitionForKeyspaceAndApply(schemaTableName, getSchemaKSKey(keyspaceName), fct);
 +    }
 +
 +    private static <T> T readSchemaPartitionForKeyspaceAndApply(String schemaTableName, ByteBuffer keyspaceKey, Function<RowIterator, T> fct)
 +    {
 +        ColumnFamilyStore store = getSchemaCFS(schemaTableName);
 +        return readSchemaPartitionForKeyspaceAndApply(store, store.decorateKey(keyspaceKey), fct);
 +    }
 +
 +    private static <T> T readSchemaPartitionForKeyspaceAndApply(String schemaTableName, DecoratedKey keyspaceKey, Function<RowIterator, T> fct)
 +    {
 +        return readSchemaPartitionForKeyspaceAndApply(getSchemaCFS(schemaTableName), keyspaceKey, fct);
 +    }
 +
 +    private static <T> T readSchemaPartitionForKeyspaceAndApply(ColumnFamilyStore store, DecoratedKey keyspaceKey, Function<RowIterator, T> fct)
 +    {
 +        int nowInSec = FBUtilities.nowInSeconds();
 +        try (OpOrder.Group op = store.readOrdering.start();
 +             RowIterator partition = UnfilteredRowIterators.filter(SinglePartitionReadCommand.fullPartitionRead(store.metadata, nowInSec, keyspaceKey)
 +                                                                                             .queryMemtableAndDisk(store, op), nowInSec))
 +        {
 +            return fct.apply(partition);
 +        }
 +    }
 +
 +    private static <T> T readSchemaPartitionForTableAndApply(String schemaTableName, String keyspaceName, String tableName, Function<RowIterator, T> fct)
 +    {
 +        ColumnFamilyStore store = getSchemaCFS(schemaTableName);
 +
 +        ClusteringComparator comparator = store.metadata.comparator;
 +        Slices slices = Slices.with(comparator, Slice.make(comparator, tableName));
 +        int nowInSec = FBUtilities.nowInSeconds();
 +        try (OpOrder.Group op = store.readOrdering.start();
 +             RowIterator partition =  UnfilteredRowIterators.filter(SinglePartitionSliceCommand.create(store.metadata, nowInSec, getSchemaKSKey(keyspaceName), slices)
 +                                                                                               .queryMemtableAndDisk(store, op), nowInSec))
 +        {
 +            return fct.apply(partition);
 +        }
 +    }
 +
 +    private static boolean isSystemKeyspaceSchemaPartition(DecoratedKey partitionKey)
 +    {
 +        return Schema.isSystemKeyspace(UTF8Type.instance.compose(partitionKey.getKey()));
 +    }
 +
 +    /**
 +     * Merge remote schema in form of mutations with local and mutate ks/cf metadata objects
 +     * (which also involves fs operations on add/drop ks/cf)
 +     *
 +     * @param mutations the schema changes to apply
 +     *
 +     * @throws ConfigurationException If one of metadata attributes has invalid value
 +     * @throws IOException If data was corrupted during transportation or failed to apply fs operations
 +     */
 +    public static synchronized void mergeSchema(Collection<Mutation> mutations) throws ConfigurationException, IOException
 +    {
 +        mergeSchema(mutations, true);
 +        Schema.instance.updateVersionAndAnnounce();
 +    }
 +
 +    public static synchronized void mergeSchema(Collection<Mutation> mutations, boolean doFlush) throws IOException
 +    {
 +        // compare before/after schemas of the affected keyspaces only
 +        Set<String> keyspaces = new HashSet<>(mutations.size());
 +        for (Mutation mutation : mutations)
 +            keyspaces.add(ByteBufferUtil.string(mutation.key().getKey()));
 +
 +        // current state of the schema
 +        Map<DecoratedKey, FilteredPartition> oldKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> oldColumnFamilies = readSchemaForKeyspaces(TABLES, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> oldViews = readSchemaForKeyspaces(VIEWS, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> oldTypes = readSchemaForKeyspaces(TYPES, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> oldFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> oldAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
 +
 +        mutations.forEach(Mutation::apply);
 +
 +        if (doFlush)
 +            flush();
 +
 +        // with new data applied
 +        Map<DecoratedKey, FilteredPartition> newKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> newColumnFamilies = readSchemaForKeyspaces(TABLES, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> newViews = readSchemaForKeyspaces(VIEWS, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> newTypes = readSchemaForKeyspaces(TYPES, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> newFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
 +        Map<DecoratedKey, FilteredPartition> newAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
 +
 +        Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
 +        mergeTables(oldColumnFamilies, newColumnFamilies);
 +        mergeViews(oldViews, newViews);
 +        mergeTypes(oldTypes, newTypes);
 +        mergeFunctions(oldFunctions, newFunctions);
 +        mergeAggregates(oldAggregates, newAggregates);
 +
 +        // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
 +        keyspacesToDrop.forEach(Schema.instance::dropKeyspace);
 +    }
 +
 +    private static Set<String> mergeKeyspaces(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after)
 +    {
 +        for (FilteredPartition newPartition : after.values())
 +        {
 +            String name = AsciiType.instance.compose(newPartition.partitionKey().getKey());
 +            KeyspaceParams params = createKeyspaceParamsFromSchemaPartition(newPartition.rowIterator());
 +
 +            FilteredPartition oldPartition = before.remove(newPartition.partitionKey());
 +            if (oldPartition == null || oldPartition.isEmpty())
 +                Schema.instance.addKeyspace(KeyspaceMetadata.create(name, params));
 +            else
 +                Schema.instance.updateKeyspace(name, params);
 +        }
 +
 +        // What's remain in old is those keyspace that are not in updated, i.e. the dropped ones.
 +        return asKeyspaceNamesSet(before.keySet());
 +    }
 +
 +    private static Set<String> asKeyspaceNamesSet(Set<DecoratedKey> keys)
 +    {
 +        Set<String> names = new HashSet<>(keys.size());
 +        for (DecoratedKey key : keys)
 +            names.add(AsciiType.instance.compose(key.getKey()));
 +        return names;
 +    }
 +
 +    private static void mergeTables(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after)
 +    {
 +        diffSchema(before, after, new Differ()
 +        {
 +            public void onDropped(UntypedResultSet.Row oldRow)
 +            {
 +                Schema.instance.dropTable(oldRow.getString("keyspace_name"), oldRow.getString("table_name"));
 +            }
 +
 +            public void onAdded(UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.addTable(createTableFromTableRow(newRow));
 +            }
 +
 +            public void onUpdated(UntypedResultSet.Row oldRow, UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.updateTable(newRow.getString("keyspace_name"), newRow.getString("table_name"));
 +            }
 +        });
 +    }
 +
 +    private static void mergeViews(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after)
 +    {
 +        diffSchema(before, after, new Differ()
 +        {
 +            public void onDropped(UntypedResultSet.Row oldRow)
 +            {
 +                Schema.instance.dropView(oldRow.getString("keyspace_name"), oldRow.getString("view_name"));
 +            }
 +
 +            public void onAdded(UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.addView(createViewFromViewRow(newRow));
 +            }
 +
 +            public void onUpdated(UntypedResultSet.Row oldRow, UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.updateView(newRow.getString("keyspace_name"), newRow.getString("view_name"));
 +            }
 +        });
 +    }
 +
 +    private static void mergeTypes(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after)
 +    {
 +        diffSchema(before, after, new Differ()
 +        {
 +            public void onDropped(UntypedResultSet.Row oldRow)
 +            {
 +                Schema.instance.dropType(createTypeFromRow(oldRow));
 +            }
 +
 +            public void onAdded(UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.addType(createTypeFromRow(newRow));
 +            }
 +
 +            public void onUpdated(UntypedResultSet.Row oldRow, UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.updateType(createTypeFromRow(newRow));
 +            }
 +        });
 +    }
 +
 +    private static void mergeFunctions(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after)
 +    {
 +        diffSchema(before, after, new Differ()
 +        {
 +            public void onDropped(UntypedResultSet.Row oldRow)
 +            {
 +                Schema.instance.dropFunction(createFunctionFromFunctionRow(oldRow));
 +            }
 +
 +            public void onAdded(UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.addFunction(createFunctionFromFunctionRow(newRow));
 +            }
 +
 +            public void onUpdated(UntypedResultSet.Row oldRow, UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.updateFunction(createFunctionFromFunctionRow(newRow));
 +            }
 +        });
 +    }
 +
 +    private static void mergeAggregates(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after)
 +    {
 +        diffSchema(before, after, new Differ()
 +        {
 +            public void onDropped(UntypedResultSet.Row oldRow)
 +            {
 +                Schema.instance.dropAggregate(createAggregateFromAggregateRow(oldRow));
 +            }
 +
 +            public void onAdded(UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.addAggregate(createAggregateFromAggregateRow(newRow));
 +            }
 +
 +            public void onUpdated(UntypedResultSet.Row oldRow, UntypedResultSet.Row newRow)
 +            {
 +                Schema.instance.updateAggregate(createAggregateFromAggregateRow(newRow));
 +            }
 +        });
 +    }
 +
 +    public interface Differ
 +    {
 +        void onDropped(UntypedResultSet.Row oldRow);
 +        void onAdded(UntypedResultSet.Row newRow);
 +        void onUpdated(UntypedResultSet.Row oldRow, UntypedResultSet.Row newRow);
 +    }
 +
 +    private static void diffSchema(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after, Differ differ)
 +    {
 +        for (FilteredPartition newPartition : after.values())
 +        {
 +            CFMetaData metadata = newPartition.metadata();
 +            DecoratedKey key = newPartition.partitionKey();
 +
 +            FilteredPartition oldPartition = before.remove(key);
 +
 +            if (oldPartition == null || oldPartition.isEmpty())
 +            {
 +                // Means everything is to be added
 +                for (Row row : newPartition)
 +                    differ.onAdded(UntypedResultSet.Row.fromInternalRow(metadata, key, row));
 +                continue;
 +            }
 +
 +            Iterator<Row> oldIter = oldPartition.iterator();
 +            Iterator<Row> newIter = newPartition.iterator();
 +
 +            Row oldRow = oldIter.hasNext() ? oldIter.next() : null;
 +            Row newRow = newIter.hasNext() ? newIter.next() : null;
 +            while (oldRow != null && newRow != null)
 +            {
 +                int cmp = metadata.comparator.compare(oldRow.clustering(), newRow.clustering());
 +                if (cmp < 0)
 +                {
 +                    differ.onDropped(UntypedResultSet.Row.fromInternalRow(metadata, key, oldRow));
 +                    oldRow = oldIter.hasNext() ? oldIter.next() : null;
 +                }
 +                else if (cmp > 0)
 +                {
 +
 +                    differ.onAdded(UntypedResultSet.Row.fromInternalRow(metadata, key, newRow));
 +                    newRow = newIter.hasNext() ? newIter.next() : null;
 +                }
 +                else
 +                {
 +                    if (!oldRow.equals(newRow))
 +                        differ.onUpdated(UntypedResultSet.Row.fromInternalRow(metadata, key, oldRow), UntypedResultSet.Row.fromInternalRow(metadata, key, newRow));
 +
 +                    oldRow = oldIter.hasNext() ? oldIter.next() : null;
 +                    newRow = newIter.hasNext() ? newIter.next() : null;
 +                }
 +            }
 +
 +            while (oldRow != null)
 +            {
 +                differ.onDropped(UntypedResultSet.Row.fromInternalRow(metadata, key, oldRow));
 +                oldRow = oldIter.hasNext() ? oldIter.next() : null;
 +            }
 +            while (newRow != null)
 +            {
 +                differ.onAdded(UntypedResultSet.Row.fromInternalRow(metadata, key, newRow));
 +                newRow = newIter.hasNext() ? newIter.next() : null;
 +            }
 +        }
 +
 +        // What remains is those keys that were only in before.
 +        for (FilteredPartition partition : before.values())
 +            for (Row row : partition)
 +                differ.onDropped(UntypedResultSet.Row.fromInternalRow(partition.metadata(), partition.partitionKey(), row));
 +    }
 +
 +    /*
 +     * Keyspace metadata serialization/deserialization.
 +     */
 +
 +    public static Mutation makeCreateKeyspaceMutation(String name, KeyspaceParams params, long timestamp)
 +    {
 +        RowUpdateBuilder adder = new RowUpdateBuilder(Keyspaces, timestamp, name).clustering();
 +        return adder.add(KeyspaceParams.Option.DURABLE_WRITES.toString(), params.durableWrites)
 +                    .frozenMap(KeyspaceParams.Option.REPLICATION.toString(), params.replication.asMap())
 +                    .build();
 +    }
 +
 +    public static Mutation makeCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
 +    {
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +
 +        keyspace.tables.forEach(table -> addTableToSchemaMutation(table, timestamp, true, mutation));
 +        keyspace.views.forEach(view -> addViewToSchemaMutation(view, timestamp, true, mutation));
 +        keyspace.types.forEach(type -> addTypeToSchemaMutation(type, timestamp, mutation));
 +        keyspace.functions.udfs().forEach(udf -> addFunctionToSchemaMutation(udf, timestamp, mutation));
 +        keyspace.functions.udas().forEach(uda -> addAggregateToSchemaMutation(uda, timestamp, mutation));
 +
 +        return mutation;
 +    }
 +
 +    public static Mutation makeDropKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
 +    {
 +        int nowInSec = FBUtilities.nowInSeconds();
 +        Mutation mutation = new Mutation(NAME, Keyspaces.decorateKey(getSchemaKSKey(keyspace.name)));
 +
 +        for (CFMetaData schemaTable : ALL_TABLE_METADATA)
 +            mutation.add(PartitionUpdate.fullPartitionDelete(schemaTable, mutation.key(), timestamp, nowInSec));
 +
 +        return mutation;
 +    }
 +
 +    private static KeyspaceMetadata createKeyspaceFromSchemaPartitions(RowIterator serializedParams,
 +                                                                       RowIterator serializedTables,
 +                                                                       RowIterator serializedViews,
 +                                                                       RowIterator serializedTypes,
 +                                                                       RowIterator serializedFunctions,
 +                                                                       RowIterator serializedAggregates)
 +    {
 +        String name = AsciiType.instance.compose(serializedParams.partitionKey().getKey());
 +
 +        KeyspaceParams params = createKeyspaceParamsFromSchemaPartition(serializedParams);
 +        Tables tables = createTablesFromTablesPartition(serializedTables);
 +        Views views = createViewsFromViewsPartition(serializedViews);
 +        Types types = createTypesFromPartition(serializedTypes);
 +
 +        Collection<UDFunction> udfs = createFunctionsFromFunctionsPartition(serializedFunctions);
 +        Collection<UDAggregate> udas = createAggregatesFromAggregatesPartition(serializedAggregates);
 +        Functions functions = org.apache.cassandra.schema.Functions.builder().add(udfs).add(udas).build();
 +
 +        return KeyspaceMetadata.create(name, params, tables, views, types, functions);
 +    }
 +
 +    /**
 +     * Deserialize only Keyspace attributes without nested tables or types
 +     *
 +     * @param partition Keyspace attributes in serialized form
 +     */
 +
 +    private static KeyspaceParams createKeyspaceParamsFromSchemaPartition(RowIterator partition)
 +    {
 +        String query = String.format("SELECT * FROM %s.%s", NAME, KEYSPACES);
 +        UntypedResultSet.Row row = QueryProcessor.resultify(query, partition).one();
 +
 +        return KeyspaceParams.create(row.getBoolean(KeyspaceParams.Option.DURABLE_WRITES.toString()),
 +                                     row.getFrozenTextMap(KeyspaceParams.Option.REPLICATION.toString()));
 +    }
 +
 +    /*
 +     * User type metadata serialization/deserialization.
 +     */
 +
 +    public static Mutation makeCreateTypeMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +        addTypeToSchemaMutation(type, timestamp, mutation);
 +        return mutation;
 +    }
 +
 +    static void addTypeToSchemaMutation(UserType type, long timestamp, Mutation mutation)
 +    {
 +        RowUpdateBuilder adder = new RowUpdateBuilder(Types, timestamp, mutation)
 +                                 .clustering(type.getNameAsString())
 +                                 .frozenList("field_names", type.fieldNames().stream().map(SchemaKeyspace::bbToString).collect(Collectors.toList()))
 +                                 .frozenList("field_types", type.fieldTypes().stream().map(AbstractType::toString).collect(Collectors.toList()));
 +
 +        adder.build();
 +    }
 +
 +    private static String bbToString(ByteBuffer bb)
 +    {
 +        try
 +        {
 +            return ByteBufferUtil.string(bb);
 +        }
 +        catch (CharacterCodingException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    public static Mutation dropTypeFromSchemaMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +        return RowUpdateBuilder.deleteRow(Types, timestamp, mutation, type.name);
 +    }
 +
 +    private static Types createTypesFromPartition(RowIterator partition)
 +    {
 +        String query = String.format("SELECT * FROM %s.%s", NAME, TYPES);
 +        Types.Builder types = org.apache.cassandra.schema.Types.builder();
 +        QueryProcessor.resultify(query, partition).forEach(row -> types.add(createTypeFromRow(row)));
 +        return types.build();
 +    }
 +
 +    private static UserType createTypeFromRow(UntypedResultSet.Row row)
 +    {
 +        String keyspace = row.getString("keyspace_name");
 +        ByteBuffer name = ByteBufferUtil.bytes(row.getString("type_name"));
 +        List<String> rawColumns = row.getFrozenList("field_names", UTF8Type.instance);
 +        List<String> rawTypes = row.getFrozenList("field_types", UTF8Type.instance);
 +
 +        List<ByteBuffer> columns = new ArrayList<>(rawColumns.size());
 +        for (String rawColumn : rawColumns)
 +            columns.add(ByteBufferUtil.bytes(rawColumn));
 +
 +        List<AbstractType<?>> types = new ArrayList<>(rawTypes.size());
 +        for (String rawType : rawTypes)
 +            types.add(parseType(rawType));
 +
 +        return new UserType(keyspace, name, columns, types);
 +    }
 +
 +    /*
 +     * Table metadata serialization/deserialization.
 +     */
 +
 +    public static Mutation makeCreateTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +        addTableToSchemaMutation(table, timestamp, true, mutation);
 +        return mutation;
 +    }
 +
 +    static void addTableToSchemaMutation(CFMetaData table, long timestamp, boolean withColumnsAndTriggers, Mutation mutation)
 +    {
 +        RowUpdateBuilder adder = new RowUpdateBuilder(Tables, timestamp, mutation).clustering(table.cfName);
 +
 +        addTableParamsToSchemaMutation(table.params, adder);
 +
 +        adder.add("id", table.cfId)
 +             .frozenSet("flags", CFMetaData.flagsToStrings(table.flags()))
 +             .build();
 +
 +        if (withColumnsAndTriggers)
 +        {
 +            for (ColumnDefinition column : table.allColumns())
 +                addColumnToSchemaMutation(table, column, timestamp, mutation);
 +
 +            for (CFMetaData.DroppedColumn column : table.getDroppedColumns().values())
 +                addDroppedColumnToSchemaMutation(table, column, timestamp, mutation);
 +
 +            for (TriggerMetadata trigger : table.getTriggers())
 +                addTriggerToSchemaMutation(table, trigger, timestamp, mutation);
 +
 +            for (IndexMetadata index : table.getIndexes())
 +                addIndexToSchemaMutation(table, index, timestamp, mutation);
 +        }
 +    }
 +
 +    private static void addTableParamsToSchemaMutation(TableParams params, RowUpdateBuilder adder)
 +    {
 +        adder.add("bloom_filter_fp_chance", params.bloomFilterFpChance)
 +             .add("comment", params.comment)
 +             .add("dclocal_read_repair_chance", params.dcLocalReadRepairChance)
 +             .add("default_time_to_live", params.defaultTimeToLive)
 +             .add("gc_grace_seconds", params.gcGraceSeconds)
 +             .add("max_index_interval", params.maxIndexInterval)
 +             .add("memtable_flush_period_in_ms", params.memtableFlushPeriodInMs)
 +             .add("min_index_interval", params.minIndexInterval)
 +             .add("read_repair_chance", params.readRepairChance)
 +             .add("speculative_retry", params.speculativeRetry.toString())
 +             .add("crc_check_chance", params.crcCheckChance)
 +             .frozenMap("caching", params.caching.asMap())
 +             .frozenMap("compaction", params.compaction.asMap())
 +             .frozenMap("compression", params.compression.asMap())
 +             .frozenMap("extensions", params.extensions);
 +    }
 +
 +    public static Mutation makeUpdateTableMutation(KeyspaceMetadata keyspace,
 +                                                   CFMetaData oldTable,
 +                                                   CFMetaData newTable,
 +                                                   long timestamp,
 +                                                   boolean fromThrift)
 +    {
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +
 +        addTableToSchemaMutation(newTable, timestamp, false, mutation);
 +
 +        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldTable.getColumnMetadata(),
 +                                                                                 newTable.getColumnMetadata());
 +
 +        // columns that are no longer needed
 +        for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
 +        {
 +            // Thrift only knows about the REGULAR ColumnDefinition type, so don't consider other type
 +            // are being deleted just because they are not here.
 +            if (!fromThrift ||
 +                column.kind == ColumnDefinition.Kind.REGULAR ||
 +                (newTable.isStaticCompactTable() && column.kind == ColumnDefinition.Kind.STATIC))
 +            {
 +                dropColumnFromSchemaMutation(oldTable, column, timestamp, mutation);
 +            }
 +        }
 +
 +        // newly added columns
 +        for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
 +            addColumnToSchemaMutation(newTable, column, timestamp, mutation);
 +
 +        // old columns with updated attributes
 +        for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
 +            addColumnToSchemaMutation(newTable, newTable.getColumnDefinition(name), timestamp, mutation);
 +
 +        // dropped columns
 +        MapDifference<ByteBuffer, CFMetaData.DroppedColumn> droppedColumnDiff =
 +            Maps.difference(oldTable.getDroppedColumns(), newTable.getDroppedColumns());
 +
 +        // newly dropped columns
 +        for (CFMetaData.DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
 +            addDroppedColumnToSchemaMutation(newTable, column, timestamp, mutation);
 +
 +        // columns added then dropped again
 +        for (ByteBuffer name : droppedColumnDiff.entriesDiffering().keySet())
 +            addDroppedColumnToSchemaMutation(newTable, newTable.getDroppedColumns().get(name), timestamp, mutation);
 +
 +        MapDifference<String, TriggerMetadata> triggerDiff = triggersDiff(oldTable.getTriggers(), newTable.getTriggers());
 +
 +        // dropped triggers
 +        for (TriggerMetadata trigger : triggerDiff.entriesOnlyOnLeft().values())
 +            dropTriggerFromSchemaMutation(oldTable, trigger, timestamp, mutation);
 +
 +        // newly created triggers
 +        for (TriggerMetadata trigger : triggerDiff.entriesOnlyOnRight().values())
 +            addTriggerToSchemaMutation(newTable, trigger, timestamp, mutation);
 +
 +        MapDifference<String, IndexMetadata> indexesDiff = indexesDiff(oldTable.getIndexes(),
 +                                                                       newTable.getIndexes());
 +
 +        // dropped indexes
 +        for (IndexMetadata index : indexesDiff.entriesOnlyOnLeft().values())
 +            dropIndexFromSchemaMutation(oldTable, index, timestamp, mutation);
 +
 +        // newly created indexes
 +        for (IndexMetadata index : indexesDiff.entriesOnlyOnRight().values())
 +            addIndexToSchemaMutation(newTable, index, timestamp, mutation);
 +
 +        // updated indexes need to be updated
 +        for (MapDifference.ValueDifference<IndexMetadata> diff : indexesDiff.entriesDiffering().values())
 +        {
 +            addUpdatedIndexToSchemaMutation(newTable, diff.rightValue(), timestamp, mutation);
 +        }
 +
 +        return mutation;
 +    }
 +
 +    private static MapDifference<String, IndexMetadata> indexesDiff(Indexes before, Indexes after)
 +    {
 +        Map<String, IndexMetadata> beforeMap = new HashMap<>();
 +        before.forEach(i -> beforeMap.put(i.name, i));
 +
 +        Map<String, IndexMetadata> afterMap = new HashMap<>();
 +        after.forEach(i -> afterMap.put(i.name, i));
 +
 +        return Maps.difference(beforeMap, afterMap);
 +    }
 +
 +    private static MapDifference<String, TriggerMetadata> triggersDiff(Triggers before, Triggers after)
 +    {
 +        Map<String, TriggerMetadata> beforeMap = new HashMap<>();
 +        before.forEach(t -> beforeMap.put(t.name, t));
 +
 +        Map<String, TriggerMetadata> afterMap = new HashMap<>();
 +        after.forEach(t -> afterMap.put(t.name, t));
 +
 +        return Maps.difference(beforeMap, afterMap);
 +    }
 +
 +    public static Mutation makeDropTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +
 +        RowUpdateBuilder.deleteRow(Tables, timestamp, mutation, table.cfName);
 +
 +        for (ColumnDefinition column : table.allColumns())
 +            dropColumnFromSchemaMutation(table, column, timestamp, mutation);
 +
 +        for (TriggerMetadata trigger : table.getTriggers())
 +            dropTriggerFromSchemaMutation(table, trigger, timestamp, mutation);
 +
 +        for (IndexMetadata index : table.getIndexes())
 +            dropIndexFromSchemaMutation(table, index, timestamp, mutation);
 +
 +        return mutation;
 +    }
 +
 +    public static CFMetaData createTableFromName(String keyspace, String table)
 +    {
 +        return readSchemaPartitionForTableAndApply(TABLES, keyspace, table, partition ->
 +        {
 +            if (partition.isEmpty())
 +                throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspace, table));
 +
 +            return createTableFromTablePartition(partition);
 +        });
 +    }
 +
 +    /**
 +     * Deserialize tables from low-level schema representation, all of them belong to the same keyspace
 +     */
 +    private static Tables createTablesFromTablesPartition(RowIterator partition)
 +    {
 +        String query = String.format("SELECT * FROM %s.%s", NAME, TABLES);
 +        Tables.Builder tables = org.apache.cassandra.schema.Tables.builder();
 +        QueryProcessor.resultify(query, partition).forEach(row -> tables.add(createTableFromTableRow(row)));
 +        return tables.build();
 +    }
 +
 +    private static List<ColumnDefinition> createColumnsFromColumnsPartition(RowIterator serializedColumns)
 +    {
 +        String query = String.format("SELECT * FROM %s.%s", NAME, COLUMNS);
 +        return createColumnsFromColumnRows(QueryProcessor.resultify(query, serializedColumns));
 +    }
 +
 +    private static CFMetaData createTableFromTablePartition(RowIterator partition)
 +    {
 +        String query = String.format("SELECT * FROM %s.%s", NAME, TABLES);
 +        return createTableFromTableRow(QueryProcessor.resultify(query, partition).one());
 +    }
 +
 +    public static CFMetaData createTableFromTablePartitionAndColumnsPartition(RowIterator tablePartition,
 +                                                                              RowIterator columnsPartition)
 +    {
 +        List<ColumnDefinition> columns = createColumnsFromColumnsPartition(columnsPartition);
 +        String query = String.format("SELECT * FROM %s.%s", NAME, TABLES);
 +        return createTableFromTableRowAndColumns(QueryProcessor.resultify(query, tablePartition).one(), columns);
 +    }
 +
 +    /**
 +     * Deserialize table metadata from low-level representation
 +     *
 +     * @return Metadata deserialized from schema
 +     */
 +    private static CFMetaData createTableFromTableRow(UntypedResultSet.Row row)
 +    {
 +        String keyspace = row.getString("keyspace_name");
 +        String table = row.getString("table_name");
 +
 +        List<ColumnDefinition> columns =
 +            readSchemaPartitionForTableAndApply(COLUMNS, keyspace, table, SchemaKeyspace::createColumnsFromColumnsPartition);
 +
 +        Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns =
 +            readSchemaPartitionForTableAndApply(DROPPED_COLUMNS, keyspace, table, SchemaKeyspace::createDroppedColumnsFromDroppedColumnsPartition);
 +
 +        Triggers triggers =
 +            readSchemaPartitionForTableAndApply(TRIGGERS, keyspace, table, SchemaKeyspace::createTriggersFromTriggersPartition);
 +
 +        CFMetaData cfm = createTableFromTableRowAndColumns(row, columns).droppedColumns(droppedColumns)
 +                                                                        .triggers(triggers);
 +
 +        // the CFMetaData itself is required to build the collection of indexes as
 +        // the column definitions are needed because we store only the name each
 +        // index's target columns and this is not enough to reconstruct a ColumnIdentifier
 +        org.apache.cassandra.schema.Indexes indexes =
 +            readSchemaPartitionForTableAndApply(INDEXES, keyspace, table, rowIterator -> createIndexesFromIndexesPartition(cfm, rowIterator));
 +        cfm.indexes(indexes);
 +
 +        return cfm;
 +    }
 +
 +    public static CFMetaData createTableFromTableRowAndColumns(UntypedResultSet.Row row, List<ColumnDefinition> columns)
 +    {
 +        String keyspace = row.getString("keyspace_name");
 +        String table = row.getString("table_name");
 +        UUID id = row.getUUID("id");
 +
 +        Set<CFMetaData.Flag> flags = row.has("flags")
 +                                   ? CFMetaData.flagsFromStrings(row.getFrozenSet("flags", UTF8Type.instance))
 +                                   : Collections.emptySet();
 +
 +        boolean isSuper = flags.contains(CFMetaData.Flag.SUPER);
 +        boolean isCounter = flags.contains(CFMetaData.Flag.COUNTER);
 +        boolean isDense = flags.contains(CFMetaData.Flag.DENSE);
 +        boolean isCompound = flags.contains(CFMetaData.Flag.COMPOUND);
 +
 +        return CFMetaData.create(keyspace,
 +                                 table,
 +                                 id,
 +                                 isDense,
 +                                 isCompound,
 +                                 isSuper,
 +                                 isCounter,
 +                                 false,
 +                                 columns,
 +                                 DatabaseDescriptor.getPartitioner())
 +                         .params(createTableParamsFromRow(row));
 +    }
 +
 +    private static TableParams createTableParamsFromRow(UntypedResultSet.Row row)
 +    {
 +        TableParams.Builder builder = TableParams.builder();
 +
 +        builder.bloomFilterFpChance(row.getDouble("bloom_filter_fp_chance"))
 +               .caching(CachingParams.fromMap(row.getFrozenTextMap("caching")))
 +               .comment(row.getString("comment"))
 +               .compaction(CompactionParams.fromMap(row.getFrozenTextMap("compaction")))
 +               .compression(CompressionParams.fromMap(row.getFrozenTextMap("compression")))
 +               .dcLocalReadRepairChance(row.getDouble("dclocal_read_repair_chance"))
 +               .defaultTimeToLive(row.getInt("default_time_to_live"))
 +               .gcGraceSeconds(row.getInt("gc_grace_seconds"))
 +               .maxIndexInterval(row.getInt("max_index_interval"))
 +               .memtableFlushPeriodInMs(row.getInt("memtable_flush_period_in_ms"))
 +               .minIndexInterval(row.getInt("min_index_interval"))
 +               .readRepairChance(row.getDouble("read_repair_chance"))
 +               .crcCheckChance(row.getDouble("crc_check_chance"))
 +               .speculativeRetry(SpeculativeRetryParam.fromString(row.getString("speculative_retry")));
 +
 +        if (row.has("extensions"))
 +            builder.extensions(row.getFrozenMap("extensions", UTF8Type.instance, BytesType.instance));
 +
 +        return builder.build();
 +    }
 +
 +    /*
 +     * Column metadata serialization/deserialization.
 +     */
 +
 +    private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
 +    {
 +        RowUpdateBuilder adder = new RowUpdateBuilder(Columns, timestamp, mutation).clustering(table.cfName, column.name.toString());
 +
 +        AbstractType<?> type = column.type;
 +        if (type instanceof ReversedType)
 +            type = ((ReversedType) type).baseType;
 +
 +        adder.add("column_name_bytes", column.name.bytes)
 +             .add("kind", column.kind.toString().toLowerCase())
 +             .add("position", column.isOnAllComponents() ? ColumnDefinition.NO_POSITION : column.position())
 +             .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
 +             .add("type", type.toString())
 +             .build();
 +    }
 +
 +    private static void dropColumnFromSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
 +    {
 +        // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
 +        RowUpdateBuilder.deleteRow(Columns, timestamp, mutation, table.cfName, column.name.toString());
 +    }
 +
 +    private static List<ColumnDefinition> createColumnsFromColumnRows(UntypedResultSet rows)
 +{
 +        List<ColumnDefinition> columns = new ArrayList<>(rows.size());
 +        rows.forEach(row -> columns.add(createColumnFromColumnRow(row)));
 +        return columns;
 +    }
 +
 +    private static ColumnDefinition createColumnFromColumnRow(UntypedResultSet.Row row)
 +    {
 +        String keyspace = row.getString("keyspace_name");
 +        String table = row.getString("table_name");
 +
 +        ColumnIdentifier name = ColumnIdentifier.getInterned(row.getBytes("column_name_bytes"), row.getString("column_name"));
 +
 +        ColumnDefinition.Kind kind = ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase());
 +
 +        int position = row.getInt("position");
 +        ClusteringOrder order = ClusteringOrder.valueOf(row.getString("clustering_order").toUpperCase());
 +
 +        AbstractType<?> type = parseType(row.getString("type"));
 +        if (order == ClusteringOrder.DESC)
 +            type = ReversedType.getInstance(type);
 +
 +        return new ColumnDefinition(keyspace, table, name, type, position, kind);
 +    }
 +
 +    /*
 +     * Dropped column metadata serialization/deserialization.
 +     */
 +
 +    private static void addDroppedColumnToSchemaMutation(CFMetaData table, CFMetaData.DroppedColumn column, long timestamp, Mutation mutation)
 +    {
 +        RowUpdateBuilder adder = new RowUpdateBuilder(DroppedColumns, timestamp, mutation).clustering(table.cfName, column.name);
 +
 +        adder.add("dropped_time", new Date(TimeUnit.MICROSECONDS.toMillis(column.droppedTime)))
 +             .add("type", column.type.toString())
 +             .build();
 +    }
 +
 +    private static Map<ByteBuffer, CFMetaData.DroppedColumn> createDroppedColumnsFromDroppedColumnsPartition(RowIterator serializedColumns)
 +    {
 +        String query = String.format("SELECT * FROM %s.%s", NAME, DROPPED_COLUMNS);
 +        Map<ByteBuffer, CFMetaData.DroppedColumn> columns = new HashMap<>();
 +        for (CFMetaData.DroppedColumn column : createDroppedColumnsFromDroppedColumnRows(QueryProcessor.resultify(query, serializedColumns)))
 +            columns.put(UTF8Type.instance.decompose(column.name), column);
 +        return columns;
 +    }
 +
 +    private static List<CFMetaData.DroppedColumn> createDroppedColumnsFromDroppedColumnRows(UntypedResultSet rows)
 +    {
 +        List<CFMetaData.DroppedColumn> columns = new ArrayList<>(rows.size());
 +        rows.forEach(row -> columns.add(createDroppedColumnFromDroppedColumnRow(row)));
 +        return columns;
 +    }
 +
 +    private static CFMetaData.DroppedColumn createDroppedColumnFromDroppedColumnRow(UntypedResultSet.Row row)
 +    {
 +        String name = row.getString("column_name");
 +        AbstractType<?> type = TypeParser.parse(row.getString("type"));
 +        long droppedTime = TimeUnit.MILLISECONDS.toMicros(row.getLong("dropped_time"));
 +
 +        return new CFMetaData.DroppedColumn(name, type, droppedTime);
 +    }
 +
 +    /*
 +     * Trigger metadata serialization/deserialization.
 +     */
 +
 +    private static void addTriggerToSchemaMutation(CFMetaData table, TriggerMetadata trigger, long timestamp, Mutation mutation)
 +    {
 +        new RowUpdateBuilder(Triggers, timestamp, mutation)
 +            .clustering(table.cfName, trigger.name)
 +            .frozenMap("options", Collections.singletonMap("class", trigger.classOption))
 +            .build();
 +    }
 +
 +    private static void dropTriggerFromSchemaMutation(CFMetaData table, TriggerMetadata trigger, long timestamp, Mutation mutation)
 +    {
 +        RowUpdateBuilder.deleteRow(Triggers, timestamp, mutation, table.cfName, trigger.name);
 +    }
 +
 +    /**
 +     * Deserialize triggers from storage-level representation.
 +     *
 +     * @param partition storage-level partition containing the trigger definitions
 +     * @return the list of processed TriggerDefinitions
 +     */
 +    private static Triggers createTriggersFromTriggersPartition(RowIterator partition)
 +    {
 +        Triggers.Builder triggers = org.apache.cassandra.schema.Triggers.builder();
 +        String query = String.format("SELECT * FROM %s.%s", NAME, TRIGGERS);
 +        QueryProcessor.resultify(query, partition).forEach(row -> triggers.add(createTriggerFromTriggerRow(row)));
 +        return triggers.build();
 +    }
 +
 +    private static TriggerMetadata createTriggerFromTriggerRow(UntypedResultSet.Row row)
 +    {
 +        String name = row.getString("trigger_name");
 +        String classOption = row.getFrozenTextMap("options").get("class");
 +        return new TriggerMetadata(name, classOption);
 +    }
 +
 +    /*
 +     * View metadata serialization/deserialization.
 +     */
 +
 +    public static Mutation makeCreateViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +        addViewToSchemaMutation(view, timestamp, true, mutation);
 +        return mutation;
 +    }
 +
 +    private static void addViewToSchemaMutation(ViewDefinition view, long timestamp, boolean includeColumns, Mutation mutation)
 +    {
 +        RowUpdateBuilder builder = new RowUpdateBuilder(Views, timestamp, mutation)
 +            .clustering(view.viewName);
 +
 +        CFMetaData table = view.metadata;
 +
 +        builder.add("include_all_columns", view.includeAllColumns)
 +               .add("base_table_id", view.baseTableId)
 +               .add("base_table_name", view.baseTableMetadata().cfName)
 +               .add("where_clause", view.whereClause)
 +               .add("id", table.cfId);
 +
 +        addTableParamsToSchemaMutation(table.params, builder);
 +
 +        if (includeColumns)
 +        {
 +            for (ColumnDefinition column : table.allColumns())
 +                addColumnToSchemaMutation(table, column, timestamp, mutation);
 +
 +            for (CFMetaData.DroppedColumn column : table.getDroppedColumns().values())
 +                addDroppedColumnToSchemaMutation(table, column, timestamp, mutation);
 +        }
 +
 +        builder.build();
 +    }
 +
 +    public static Mutation makeDropViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +
 +        RowUpdateBuilder.deleteRow(Views, timestamp, mutation, view.viewName);
 +
 +        CFMetaData table = view.metadata;
 +        for (ColumnDefinition column : table.allColumns())
 +            dropColumnFromSchemaMutation(table, column, timestamp, mutation);
 +
 +        for (IndexMetadata index : table.getIndexes())
 +            dropIndexFromSchemaMutation(table, index, timestamp, mutation);
 +
 +        return mutation;
 +    }
 +
 +    public static Mutation makeUpdateViewMutation(KeyspaceMetadata keyspace,
 +                                                  ViewDefinition oldView,
 +                                                  ViewDefinition newView,
 +                                                  long timestamp)
 +    {
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +
 +        addViewToSchemaMutation(newView, timestamp, false, mutation);
 +
 +        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldView.metadata.getColumnMetadata(),
 +                                                                                 newView.metadata.getColumnMetadata());
 +
 +        // columns that are no longer needed
 +        for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
 +        {
 +            dropColumnFromSchemaMutation(oldView.metadata, column, timestamp, mutation);
 +        }
 +
 +        // newly added columns
 +        for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
 +            addColumnToSchemaMutation(newView.metadata, column, timestamp, mutation);
 +
 +        // old columns with updated attributes
 +        for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
 +            addColumnToSchemaMutation(newView.metadata, newView.metadata.getColumnDefinition(name), timestamp, mutation);
 +
 +        // dropped columns
 +        MapDifference<ByteBuffer, CFMetaData.DroppedColumn> droppedColumnDiff =
 +        Maps.difference(oldView.metadata.getDroppedColumns(), oldView.metadata.getDroppedColumns());
 +
 +        // newly dropped columns
 +        for (CFMetaData.DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
 +            addDroppedColumnToSchemaMutation(oldView.metadata, column, timestamp, mutation);
 +
 +        // columns added then dropped again
 +        for (ByteBuffer name : droppedColumnDiff.entriesDiffering().keySet())
 +            addDroppedColumnToSchemaMutation(newView.metadata, newView.metadata.getDroppedColumns().get(name), timestamp, mutation);
 +
 +        return mutation;
 +    }
 +
 +    public static ViewDefinition createViewFromName(String keyspace, String view)
 +    {
 +        return readSchemaPartitionForTableAndApply(VIEWS, keyspace, view, partition ->
 +        {
 +            if (partition.isEmpty())
 +                throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspace, view));
 +
 +            return createViewFromViewPartition(partition);
 +        });
 +    }
 +
 +    private static ViewDefinition createViewFromViewPartition(RowIterator partition)
 +    {
 +        String query = String.format("SELECT * FROM %s.%s", NAME, VIEWS);
 +        return createViewFromViewRow(QueryProcessor.resultify(query, partition).one());
 +    }
 +
 +    /**
 +     * Deserialize views from storage-level representation.
 +     *
 +     * @param partition storage-level partition containing the view definitions
 +     * @return the list of processed ViewDefinitions
 +     */
 +    private static Views createViewsFromViewsPartition(RowIterator partition)
 +    {
 +        Views.Builder views = org.apache.cassandra.schema.Views.builder();
 +        String query = String.format("SELECT * FROM %s.%s", NAME, VIEWS);
 +        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
 +        {
 +            ViewDefinition view = createViewFromViewRow(row);
 +            views.add(view);
 +        }
 +        return views.build();
 +    }
 +
 +    private static ViewDefinition createViewFromViewRow(UntypedResultSet.Row row)
 +    {
 +        String keyspace = row.getString("keyspace_name");
 +        String view = row.getString("view_name");
 +        UUID id = row.getUUID("id");
 +        UUID baseTableId = row.getUUID("base_table_id");
 +        String baseTableName = row.getString("base_table_name");
 +        boolean includeAll = row.getBoolean("include_all_columns");
 +        String whereClause = row.getString("where_clause");
 +
 +        List<ColumnDefinition> columns =
 +            readSchemaPartitionForTableAndApply(COLUMNS, keyspace, view, SchemaKeyspace::createColumnsFromColumnsPartition);
 +
 +        Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns =
 +            readSchemaPartitionForTableAndApply(DROPPED_COLUMNS, keyspace, view, SchemaKeyspace::createDroppedColumnsFromDroppedColumnsPartition);
 +
 +        CFMetaData cfm = CFMetaData.create(keyspace,
 +                                           view,
 +                                           id,
 +                                           false,
 +                                           true,
 +                                           false,
 +                                           false,
 +                                           true,
 +                                           columns,
 +                                           DatabaseDescriptor.getPartitioner())
 +                                   .params(createTableParamsFromRow(row))
 +                                   .droppedColumns(droppedColumns);
 +
 +        String rawSelect = View.buildSelectStatement(baseTableName, columns, whereClause);
 +        SelectStatement.RawStatement rawStatement = (SelectStatement.RawStatement) QueryProcessor.parseStatement(rawSelect);
 +
 +        return new ViewDefinition(keyspace, view, baseTableId, baseTableName, includeAll, rawStatement, whereClause, cfm);
 +    }
 +
 +    /*
 +     * Secondary Index metadata serialization/deserialization.
 +     */
 +
 +    private static void addIndexToSchemaMutation(CFMetaData table,
 +                                                 IndexMetadata index,
 +                                                 long timestamp,
 +                                                 Mutation mutation)
 +    {
 +        RowUpdateBuilder builder = new RowUpdateBuilder(Indexes, timestamp, mutation).clustering(table.cfName, index.name);
 +
 +        builder.add("kind", index.kind.toString());
 +        builder.frozenMap("options", index.options);
 +        builder.build();
 +    }
 +
 +    private static void dropIndexFromSchemaMutation(CFMetaData table,
 +                                                    IndexMetadata index,
 +                                                    long timestamp,
 +                                                    Mutation mutation)
 +    {
 +        RowUpdateBuilder.deleteRow(Indexes, timestamp, mutation, table.cfName, index.name);
 +    }
 +
 +    private static void addUpdatedIndexToSchemaMutation(CFMetaData table,
 +                                                        IndexMetadata index,
 +                                                        long timestamp,
 +                                                        Mutation mutation)
 +    {
 +        addIndexToSchemaMutation(table, index, timestamp, mutation);
 +    }
 +    /**
 +     * Deserialize secondary indexes from storage-level representation.
 +     *
 +     * @param partition storage-level partition containing the index definitions
 +     * @return the list of processed IndexMetadata
 +     */
 +    private static Indexes createIndexesFromIndexesPartition(CFMetaData cfm, RowIterator partition)
 +    {
 +        Indexes.Builder indexes = org.apache.cassandra.schema.Indexes.builder();
 +        String query = String.format("SELECT * FROM %s.%s", NAME, INDEXES);
 +        QueryProcessor.resultify(query, partition).forEach(row -> indexes.add(createIndexMetadataFromIndexesRow(row)));
 +        return indexes.build();
 +    }
 +
 +    private static IndexMetadata createIndexMetadataFromIndexesRow(UntypedResultSet.Row row)
 +    {
 +        String name = row.getString("index_name");
 +        IndexMetadata.Kind type = IndexMetadata.Kind.valueOf(row.getString("kind"));
 +        Map<String, String> options = row.getFrozenTextMap("options");
 +        return IndexMetadata.fromSchemaMetadata(name, type, options);
 +    }
 +
 +    /*
 +     * UDF metadata serialization/deserialization.
 +     */
 +
 +    public static Mutation makeCreateFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +        addFunctionToSchemaMutation(function, timestamp, mutation);
 +        return mutation;
 +    }
 +
 +    static void addFunctionToSchemaMutation(UDFunction function, long timestamp, Mutation mutation)
 +    {
 +        RowUpdateBuilder adder = new RowUpdateBuilder(Functions, timestamp, mutation)
 +                                 .clustering(function.name().name, functionSignatureWithTypes(function));
 +
 +        adder.add("body", function.body())
 +             .add("language", function.language())
 +             .add("return_type", function.returnType().toString())
 +             .add("called_on_null_input", function.isCalledOnNullInput())
 +             .frozenList("argument_names", function.argNames().stream().map((c) -> bbToString(c.bytes)).collect(Collectors.toList()))
 +             .frozenList("argument_types", function.argTypes().stream().map(AbstractType::toString).collect(Collectors.toList()));
 +
 +        adder.build();
 +    }
 +
 +    public static Mutation makeDropFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +        return RowUpdateBuilder.deleteRow(Functions, timestamp, mutation, function.name().name, functionSignatureWithTypes(function));
 +    }
 +
 +    private static Collection<UDFunction> createFunctionsFromFunctionsPartition(RowIterator partition)
 +    {
 +        List<UDFunction> functions = new ArrayList<>();
 +        String query = String.format("SELECT * FROM %s.%s", NAME, FUNCTIONS);
 +        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
 +            functions.add(createFunctionFromFunctionRow(row));
 +        return functions;
 +    }
 +
 +    private static UDFunction createFunctionFromFunctionRow(UntypedResultSet.Row row)
 +    {
 +        String ksName = row.getString("keyspace_name");
 +        String functionName = row.getString("function_name");
 +        FunctionName name = new FunctionName(ksName, functionName);
 +
 +        List<ColumnIdentifier> argNames = new ArrayList<>();
 +        if (row.has("argument_names"))
 +            for (String arg : row.getFrozenList("argument_names", UTF8Type.instance))
 +                argNames.add(new ColumnIdentifier(arg, true));
 +
 +        List<AbstractType<?>> argTypes = new ArrayList<>();
 +        if (row.has("argument_types"))
 +            for (String type : row.getFrozenList("argument_types", UTF8Type.instance))
 +                argTypes.add(parseType(type));
 +
 +        AbstractType<?> returnType = parseType(row.getString("return_type"));
 +
 +        String language = row.getString("language");
 +        String body = row.getString("body");
 +        boolean calledOnNullInput = row.getBoolean("called_on_null_input");
 +
 +        org.apache.cassandra.cql3.functions.Function existing = Schema.instance.findFunction(name, argTypes).orElse(null);
 +        if (existing instanceof UDFunction)
 +        {
 +            // This check prevents duplicate compilation of effectively the same UDF.
 +            // Duplicate compilation attempts can occur on the coordinator node handling the CREATE FUNCTION
 +            // statement, since CreateFunctionStatement needs to execute UDFunction.create but schema migration
 +            // also needs that (since it needs to handle its own change).
 +            UDFunction udf = (UDFunction) existing;
 +            if (udf.argNames().equals(argNames) && // arg types checked in Functions.find call
 +                udf.returnType().equals(returnType) &&
 +                !udf.isAggregate() &&
 +                udf.language().equals(language) &&
 +                udf.body().equals(body) &&
 +                udf.isCalledOnNullInput() == calledOnNullInput)
 +            {
-                 logger.debug("Skipping duplicate compilation of already existing UDF {}", name);
++                logger.trace("Skipping duplicate compilation of already existing UDF {}", name);
 +                return udf;
 +            }
 +        }
 +
 +        try
 +        {
 +            return UDFunction.create(name, argNames, argTypes, returnType, calledOnNullInput, language, body);
 +        }
 +        catch (InvalidRequestException e)
 +        {
 +            logger.error(String.format("Cannot load function '%s' from schema: this function won't be available (on this node)", name), e);
 +            return UDFunction.createBrokenFunction(name, argNames, argTypes, returnType, calledOnNullInput, language, body, e);
 +        }
 +    }
 +
 +    /*
 +     * Aggregate UDF metadata serialization/deserialization.
 +     */
 +
 +    public static Mutation makeCreateAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +        addAggregateToSchemaMutation(aggregate, timestamp, mutation);
 +        return mutation;
 +    }
 +
 +    static void addAggregateToSchemaMutation(UDAggregate aggregate, long timestamp, Mutation mutation)
 +    {
 +        RowUpdateBuilder adder = new RowUpdateBuilder(Aggregates, timestamp, mutation)
 +                                 .clustering(aggregate.name().name, functionSignatureWithTypes(aggregate));
 +
 +        adder.add("return_type", aggregate.returnType().toString())
 +             .add("state_func", aggregate.stateFunction().name().name)
 +             .add("state_type", aggregate.stateType() != null ? aggregate.stateType().toString() : null)
 +             .add("final_func", aggregate.finalFunction() != null ? aggregate.finalFunction().name().name : null)
 +             .add("initcond", aggregate.initialCondition())
 +             .frozenList("argument_types", aggregate.argTypes().stream().map(AbstractType::toString).collect(Collectors.toList()))
 +             .build();
 +    }
 +
 +    private static Collection<UDAggregate> createAggregatesFromAggregatesPartition(RowIterator partition)
 +    {
 +        List<UDAggregate> aggregates = new ArrayList<>();
 +        String query = String.format("SELECT * FROM %s.%s", NAME, AGGREGATES);
 +        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
 +            aggregates.add(createAggregateFromAggregateRow(row));
 +        return aggregates;
 +    }
 +
 +    private static UDAggregate createAggregateFromAggregateRow(UntypedResultSet.Row row)
 +    {
 +        String ksName = row.getString("keyspace_name");
 +        String functionName = row.getString("aggregate_name");
 +        FunctionName name = new FunctionName(ksName, functionName);
 +
 +        List<String> types = row.getFrozenList("argument_types", UTF8Type.instance);
 +
 +        List<AbstractType<?>> argTypes;
 +        if (types == null)
 +        {
 +            argTypes = Collections.emptyList();
 +        }
 +        else
 +        {
 +            argTypes = new ArrayList<>(types.size());
 +            for (String type : types)
 +                argTypes.add(parseType(type));
 +        }
 +
 +        AbstractType<?> returnType = parseType(row.getString("return_type"));
 +
 +        FunctionName stateFunc = new FunctionName(ksName, (row.getString("state_func")));
 +        FunctionName finalFunc = row.has("final_func") ? new FunctionName(ksName, row.getString("final_func")) : null;
 +        AbstractType<?> stateType = row.has("state_type") ? parseType(row.getString("state_type")) : null;
 +        ByteBuffer initcond = row.has("initcond") ? row.getBytes("initcond") : null;
 +
 +        try
 +        {
 +            return UDAggregate.create(name, argTypes, returnType, stateFunc, finalFunc, stateType, initcond);
 +        }
 +        catch (InvalidRequestException reason)
 +        {
 +            return UDAggregate.createBroken(name, argTypes, returnType, initcond, reason);
 +        }
 +    }
 +
 +    public static Mutation makeDropAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
 +    {
 +        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
 +        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 +        return RowUpdateBuilder.deleteRow(Aggregates, timestamp, mutation, aggregate.name().name, functionSignatureWithTypes(aggregate));
 +    }
 +
 +    private static AbstractType<?> parseType(String str)
 +    {
 +        return TypeParser.parse(str);
 +    }
 +
 +    // We allow method overloads, so a function is not uniquely identified by its name only, but
 +    // also by its argument types. To distinguish overloads of given function name in the schema
 +    // we use a "signature" which is just a list of it's CQL argument types (we could replace that by
 +    // using a "signature" UDT that would be comprised of the function name and argument types,
 +    // which we could then use as clustering column. But as we haven't yet used UDT in system tables,
 +    // We'll leave that decision to #6717).
 +    public static ByteBuffer functionSignatureWithTypes(AbstractFunction fun)
 +    {
 +        ListType<String> list = ListType.getInstance(UTF8Type.instance, false);
 +        List<String> strList = new ArrayList<>(fun.argTypes().size());
 +        for (AbstractType<?> argType : fun.argTypes())
 +            strList.add(argType.asCQL3Type().toString());
 +        return list.decompose(strList);
 +    }
 +}


[7/9] cassandra git commit: Merge branch cassandra-2.2 into cassandra-3.0

Posted by bl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Keyspace.java
index 0676e45,92a0950..cf34e9a
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@@ -302,16 -264,14 +302,16 @@@ public class Keyspac
          createReplicationStrategy(metadata);
  
          this.metric = new KeyspaceMetrics(this);
 -        for (CFMetaData cfm : new ArrayList<>(metadata.cfMetaData().values()))
 +        this.viewManager = new ViewManager(this);
 +        for (CFMetaData cfm : metadata.tablesAndViews())
          {
-             logger.debug("Initializing {}.{}", getName(), cfm.cfName);
+             logger.trace("Initializing {}.{}", getName(), cfm.cfName);
              initCf(cfm.cfId, cfm.cfName, loadSSTables);
          }
 +        this.viewManager.reload();
      }
  
 -    private Keyspace(KSMetaData metadata)
 +    private Keyspace(KeyspaceMetadata metadata)
      {
          this.metadata = metadata;
          createReplicationStrategy(metadata);
@@@ -415,37 -379,6 +415,37 @@@
          if (TEST_FAIL_WRITES && metadata.name.equals(TEST_FAIL_WRITES_KS))
              throw new RuntimeException("Testing write failures");
  
 +        Lock lock = null;
 +        boolean requiresViewUpdate = updateIndexes && viewManager.updatesAffectView(Collections.singleton(mutation), false);
 +
 +        if (requiresViewUpdate)
 +        {
 +            lock = ViewManager.acquireLockFor(mutation.key().getKey());
 +
 +            if (lock == null)
 +            {
 +                if ((System.currentTimeMillis() - mutation.createdAt) > DatabaseDescriptor.getWriteRpcTimeout())
 +                {
-                     logger.debug("Could not acquire lock for {}", ByteBufferUtil.bytesToHex(mutation.key().getKey()));
++                    logger.trace("Could not acquire lock for {}", ByteBufferUtil.bytesToHex(mutation.key().getKey()));
 +                    Tracing.trace("Could not acquire MV lock");
 +                    throw new WriteTimeoutException(WriteType.VIEW, ConsistencyLevel.LOCAL_ONE, 0, 1);
 +                }
 +                else
 +                {
 +                    //This view update can't happen right now. so rather than keep this thread busy
 +                    // we will re-apply ourself to the queue and try again later
 +                    StageManager.getStage(Stage.MUTATION).execute(() -> {
 +                        if (writeCommitLog)
 +                            mutation.apply();
 +                        else
 +                            mutation.applyUnsafe();
 +                    });
 +
 +                    return;
 +                }
 +            }
 +        }
 +        int nowInSec = FBUtilities.nowInSeconds();
          try (OpOrder.Group opGroup = writeOrder.start())
          {
              // write the mutation to the commitlog and memtables
@@@ -502,22 -415,30 +502,22 @@@
  
      /**
       * @param key row to index
 -     * @param cfs ColumnFamily to index row in
 -     * @param idxNames columns to index, in comparator order
 +     * @param cfs ColumnFamily to index partition in
 +     * @param indexes the indexes to submit the row to
       */
 -    public static void indexRow(DecoratedKey key, ColumnFamilyStore cfs, Set<String> idxNames)
 +    public static void indexPartition(DecoratedKey key, ColumnFamilyStore cfs, Set<Index> indexes)
      {
-         if (logger.isDebugEnabled())
-             logger.debug("Indexing partition {} ", cfs.metadata.getKeyValidator().getString(key.getKey()));
+         if (logger.isTraceEnabled())
 -            logger.trace("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.getKey()));
++            logger.trace("Indexing partition {} ", cfs.metadata.getKeyValidator().getString(key.getKey()));
  
 -        try (OpOrder.Group opGroup = cfs.keyspace.writeOrder.start())
 -        {
 -            Set<SecondaryIndex> indexes = cfs.indexManager.getIndexesByNames(idxNames);
 +        SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(cfs.metadata,
 +                                                                                      FBUtilities.nowInSeconds(),
 +                                                                                      key);
  
 -            Iterator<ColumnFamily> pager = QueryPagers.pageRowLocally(cfs, key.getKey(), DEFAULT_PAGE_SIZE);
 -            while (pager.hasNext())
 -            {
 -                ColumnFamily cf = pager.next();
 -                ColumnFamily cf2 = cf.cloneMeShallow();
 -                for (Cell cell : cf)
 -                {
 -                    if (cfs.indexManager.indexes(cell.name(), indexes))
 -                        cf2.addColumn(cell);
 -                }
 -                cfs.indexManager.indexRow(key.getKey(), cf2, opGroup);
 -            }
 +        try (OpOrder.Group opGroup = cfs.keyspace.writeOrder.start();
 +             UnfilteredRowIterator partition = cmd.queryMemtableAndDisk(cfs, opGroup))
 +        {
 +            cfs.indexManager.indexPartition(partition, opGroup, indexes, cmd.nowInSec());
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Memtable.java
index 7af65d1,e96a71e..81decde
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@@ -354,17 -355,18 +354,17 @@@ public class Memtable implements Compar
  
          protected Directories getDirectories()
          {
 -            return cfs.directories;
 +            return cfs.getDirectories();
          }
  
 -        private SSTableReader writeSortedContents(ReplayPosition context, File sstableDirectory)
 +        private Collection<SSTableReader> writeSortedContents(ReplayPosition context, File sstableDirectory)
          {
-             logger.info("Writing {}", Memtable.this.toString());
+             logger.debug("Writing {}", Memtable.this.toString());
  
 -            SSTableReader ssTable;
 -            // errors when creating the writer that may leave empty temp files.
 -            try (SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory)))
 +            Collection<SSTableReader> ssTables;
 +            try (SSTableTxnWriter writer = createFlushWriter(cfs.getSSTablePath(sstableDirectory), columnsCollector.get(), statsCollector.get()))
              {
-                 boolean trackContention = logger.isDebugEnabled();
+                 boolean trackContention = logger.isTraceEnabled();
                  int heavilyContendedRowCount = 0;
                  // (we can't clear out the map as-we-go to free up memory,
                  //  since the memtable is being used for queries in the "pending flush" category)
@@@ -392,26 -394,26 +392,26 @@@
  
                  if (writer.getFilePointer() > 0)
                  {
-                     logger.info(String.format("Completed flushing %s (%s) for commitlog position %s",
-                                               writer.getFilename(),
-                                               FBUtilities.prettyPrintMemory(writer.getFilePointer()),
-                                               context));
+                     logger.debug(String.format("Completed flushing %s (%s) for commitlog position %s",
 -                                              writer.getFilename(),
 -                                              FBUtilities.prettyPrintMemory(writer.getOnDiskFilePointer()),
 -                                              context));
++                                               writer.getFilename(),
++                                               FBUtilities.prettyPrintMemory(writer.getFilePointer()),
++                                               context));
  
 -                    // temp sstables should contain non-repaired data.
 -                    ssTable = writer.finish(true);
 +                    // sstables should contain non-repaired data.
 +                    ssTables = writer.finish(true);
                  }
                  else
                  {
-                     logger.info("Completed flushing {}; nothing needed to be retained.  Commitlog position was {}",
+                     logger.debug("Completed flushing {}; nothing needed to be retained.  Commitlog position was {}",
                                  writer.getFilename(), context);
                      writer.abort();
 -                    ssTable = null;
 +                    ssTables = null;
                  }
  
                  if (heavilyContendedRowCount > 0)
-                     logger.debug(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, partitions.size(), Memtable.this.toString()));
 -                    logger.trace(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, rows.size(), Memtable.this.toString()));
++                    logger.trace(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, partitions.size(), Memtable.this.toString()));
  
 -                return ssTable;
 +                return ssTables;
              }
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
index 6502ed3,ab934c6..3666b27
--- a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
@@@ -39,9 -39,9 +39,9 @@@ public class MigrationRequestVerbHandle
  
      public void doVerb(MessageIn message, int id)
      {
-         logger.debug("Received migration request from {}.", message.from);
+         logger.trace("Received migration request from {}.", message.from);
          MessageOut<Collection<Mutation>> response = new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE,
 -                                                                     LegacySchemaTables.convertSchemaToMutations(),
 +                                                                     SchemaKeyspace.convertSchemaToMutations(),
                                                                       MigrationManager.MigrationsSerializer.instance);
          MessagingService.instance().sendReply(response, id, message.from);
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/SystemKeyspace.java
index 6b2585e,308edcd..5c47e1e
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@@ -1320,36 -1122,6 +1320,36 @@@ public final class SystemKeyspac
          return result.one().getString("release_version");
      }
  
 +    /**
 +     * Check data directories for old files that can be removed when migrating from 2.1 or 2.2 to 3.0,
 +     * these checks can be removed in 4.0, see CASSANDRA-7066
 +     */
 +    public static void migrateDataDirs()
 +    {
 +        Iterable<String> dirs = Arrays.asList(DatabaseDescriptor.getAllDataFileLocations());
 +        for (String dataDir : dirs)
 +        {
-             logger.debug("Checking directory {} for old files", dataDir);
++            logger.trace("Checking directory {} for old files", dataDir);
 +            File dir = new File(dataDir);
 +            assert dir.exists() : dir + " should have been created by startup checks";
 +
 +            for (File ksdir : dir.listFiles((d, n) -> d.isDirectory()))
 +            {
 +                for (File cfdir : ksdir.listFiles((d, n) -> d.isDirectory()))
 +                {
 +                    if (Descriptor.isLegacyFile(cfdir))
 +                    {
 +                        FileUtils.deleteRecursive(cfdir);
 +                    }
 +                    else
 +                    {
 +                        FileUtils.delete(cfdir.listFiles((d, n) -> Descriptor.isLegacyFile(new File(d, n))));
 +                    }
 +                }
 +            }
 +        }
 +    }
 +
      private static ByteBuffer rangeToBytes(Range<Token> range)
      {
          try (DataOutputBuffer out = new DataOutputBuffer())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index 7049191,cb02a8c..2668bba
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@@ -404,7 -388,11 +404,7 @@@ public class CommitLogReplaye
                  if (!replaySyncSection(sectionReader, replayEnd, desc, errorContext, tolerateErrorsInSection))
                      break;
              }
-             logger.info("Finished reading {}", file);
 -        }
 -        finally
 -        {
 -            FileUtils.closeQuietly(reader);
+             logger.debug("Finished reading {}", file);
          }
      }
  
@@@ -562,8 -551,8 +562,8 @@@
              return;
          }
  
-         if (logger.isDebugEnabled())
-             logger.debug("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), mutation.key(), "{" + StringUtils.join(mutation.getPartitionUpdates().iterator(), ", ") + "}");
+         if (logger.isTraceEnabled())
 -            logger.trace("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), ByteBufferUtil.bytesToHex(mutation.key()), "{" + StringUtils.join(mutation.getColumnFamilies().iterator(), ", ") + "}");
++            logger.trace("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), mutation.key(), "{" + StringUtils.join(mutation.getPartitionUpdates().iterator(), ", ") + "}");
  
          Runnable runnable = new WrappedRunnable()
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 75d50e7,ea20a1f..77c0cbb
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -157,10 -165,10 +157,10 @@@ public class CompactionManager implemen
              return Collections.emptyList();
          }
  
-         logger.debug("Scheduling a background task check for {}.{} with {}",
+         logger.trace("Scheduling a background task check for {}.{} with {}",
                       cfs.keyspace.getName(),
                       cfs.name,
 -                     cfs.getCompactionStrategy().getName());
 +                     cfs.getCompactionStrategyManager().getName());
          List<Future<?>> futures = new ArrayList<>();
          // we must schedule it at least once, otherwise compaction will stop for a CF until next flush
          if (executor.isShutdown())
@@@ -210,11 -218,11 +210,11 @@@
                      return;
                  }
  
 -                AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
 -                AbstractCompactionTask task = strategy.getNextBackgroundTask(getDefaultGcBefore(cfs));
 +                CompactionStrategyManager strategy = cfs.getCompactionStrategyManager();
 +                AbstractCompactionTask task = strategy.getNextBackgroundTask(getDefaultGcBefore(cfs, FBUtilities.nowInSeconds()));
                  if (task == null)
                  {
-                     logger.debug("No tasks available");
+                     logger.trace("No tasks available");
                      return;
                  }
                  task.execute(metrics);
@@@ -452,8 -460,8 +452,8 @@@
                                        LifecycleTransaction txn,
                                        long repairedAt) throws InterruptedException, IOException
      {
 -        logger.info("Starting anticompaction for {}.{} on {}/{} sstables", cfs.keyspace.getName(), cfs.getColumnFamilyName(), validatedForRepair.size(), cfs.getSSTables().size());
 +        logger.info("Starting anticompaction for {}.{} on {}/{} sstables", cfs.keyspace.getName(), cfs.getColumnFamilyName(), validatedForRepair.size(), cfs.getLiveSSTables());
-         logger.debug("Starting anticompaction for ranges {}", ranges);
+         logger.trace("Starting anticompaction for ranges {}", ranges);
          Set<SSTableReader> sstables = new HashSet<>(validatedForRepair);
          Set<SSTableReader> mutatedRepairStatuses = new HashSet<>();
          Set<SSTableReader> nonAnticompacting = new HashSet<>();
@@@ -781,10 -788,10 +781,10 @@@
  
          long totalkeysWritten = 0;
  
 -        long expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(),
 +        long expectedBloomFilterSize = Math.max(cfs.metadata.params.minIndexInterval,
                                                 SSTableReader.getApproximateKeyCount(txn.originals()));
-         if (logger.isDebugEnabled())
-             logger.debug("Expected bloom filter size : {}", expectedBloomFilterSize);
+         if (logger.isTraceEnabled())
+             logger.trace("Expected bloom filter size : {}", expectedBloomFilterSize);
  
          logger.info("Cleaning up {}", sstable);
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 47c8de8,0000000..bd72c64
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@@ -1,513 -1,0 +1,513 @@@
 +/*
 + * 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.db.compaction;
 +
 +
 +import java.util.*;
 +import java.util.concurrent.Callable;
 +
 +import com.google.common.collect.Iterables;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.Memtable;
 +import org.apache.cassandra.db.SerializationHeader;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.dht.Range;
 +import org.apache.cassandra.dht.Token;
 +import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.ISSTableScanner;
 +import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 +import org.apache.cassandra.notifications.*;
 +import org.apache.cassandra.schema.CompactionParams;
 +import org.apache.cassandra.service.ActiveRepairService;
 +
 +/**
 + * Manages the compaction strategies.
 + *
 + * Currently has two instances of actual compaction strategies - one for repaired data and one for
 + * unrepaired data. This is done to be able to totally separate the different sets of sstables.
 + */
 +public class CompactionStrategyManager implements INotificationConsumer
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(CompactionStrategyManager.class);
 +    private final ColumnFamilyStore cfs;
 +    private volatile AbstractCompactionStrategy repaired;
 +    private volatile AbstractCompactionStrategy unrepaired;
 +    private volatile boolean enabled = true;
 +    public boolean isActive = true;
 +    private volatile CompactionParams params;
 +    /*
 +        We keep a copy of the schema compaction parameters here to be able to decide if we
 +        should update the compaction strategy in maybeReloadCompactionStrategy() due to an ALTER.
 +
 +        If a user changes the local compaction strategy and then later ALTERs a compaction parameter,
 +        we will use the new compaction parameters.
 +     */
 +    private CompactionParams schemaCompactionParams;
 +
 +    public CompactionStrategyManager(ColumnFamilyStore cfs)
 +    {
 +        cfs.getTracker().subscribe(this);
-         logger.debug("{} subscribed to the data tracker.", this);
++        logger.trace("{} subscribed to the data tracker.", this);
 +        this.cfs = cfs;
 +        reload(cfs.metadata);
 +        params = cfs.metadata.params.compaction;
 +        enabled = params.isEnabled();
 +    }
 +
 +    /**
 +     * Return the next background task
 +     *
 +     * Returns a task for the compaction strategy that needs it the most (most estimated remaining tasks)
 +     *
 +     */
 +    public synchronized AbstractCompactionTask getNextBackgroundTask(int gcBefore)
 +    {
 +        if (!isEnabled())
 +            return null;
 +
 +        maybeReload(cfs.metadata);
 +
 +        if (repaired.getEstimatedRemainingTasks() > unrepaired.getEstimatedRemainingTasks())
 +        {
 +            AbstractCompactionTask repairedTask = repaired.getNextBackgroundTask(gcBefore);
 +            if (repairedTask != null)
 +                return repairedTask;
 +            return unrepaired.getNextBackgroundTask(gcBefore);
 +        }
 +        else
 +        {
 +            AbstractCompactionTask unrepairedTask = unrepaired.getNextBackgroundTask(gcBefore);
 +            if (unrepairedTask != null)
 +                return unrepairedTask;
 +            return repaired.getNextBackgroundTask(gcBefore);
 +        }
 +    }
 +
 +    public boolean isEnabled()
 +    {
 +        return enabled && isActive;
 +    }
 +
 +    public synchronized void resume()
 +    {
 +        isActive = true;
 +    }
 +
 +    /**
 +     * pause compaction while we cancel all ongoing compactions
 +     *
 +     * Separate call from enable/disable to not have to save the enabled-state externally
 +      */
 +    public synchronized void pause()
 +    {
 +        isActive = false;
 +    }
 +
 +
 +    private void startup()
 +    {
 +        for (SSTableReader sstable : cfs.getSSTables(SSTableSet.CANONICAL))
 +        {
 +            if (sstable.openReason != SSTableReader.OpenReason.EARLY)
 +                getCompactionStrategyFor(sstable).addSSTable(sstable);
 +        }
 +        repaired.startup();
 +        unrepaired.startup();
 +    }
 +
 +    /**
 +     * return the compaction strategy for the given sstable
 +     *
 +     * returns differently based on the repaired status
 +     * @param sstable
 +     * @return
 +     */
 +    private AbstractCompactionStrategy getCompactionStrategyFor(SSTableReader sstable)
 +    {
 +        if (sstable.isRepaired())
 +            return repaired;
 +        else
 +            return unrepaired;
 +    }
 +
 +    public void shutdown()
 +    {
 +        isActive = false;
 +        repaired.shutdown();
 +        unrepaired.shutdown();
 +    }
 +
 +    public synchronized void maybeReload(CFMetaData metadata)
 +    {
 +        // compare the old schema configuration to the new one, ignore any locally set changes.
 +        if (metadata.params.compaction.equals(schemaCompactionParams))
 +            return;
 +        reload(metadata);
 +    }
 +
 +    /**
 +     * Reload the compaction strategies
 +     *
 +     * Called after changing configuration and at startup.
 +     * @param metadata
 +     */
 +    public synchronized void reload(CFMetaData metadata)
 +    {
 +        boolean disabledWithJMX = !enabled && shouldBeEnabled();
 +        setStrategy(metadata.params.compaction);
 +        schemaCompactionParams = metadata.params.compaction;
 +
 +        if (disabledWithJMX || !shouldBeEnabled())
 +            disable();
 +        else
 +            enable();
 +        startup();
 +    }
 +
 +    public void replaceFlushed(Memtable memtable, Collection<SSTableReader> sstables)
 +    {
 +        cfs.getTracker().replaceFlushed(memtable, sstables);
 +        if (sstables != null && !sstables.isEmpty())
 +            CompactionManager.instance.submitBackground(cfs);
 +    }
 +
 +    public int getUnleveledSSTables()
 +    {
 +        if (repaired instanceof LeveledCompactionStrategy && unrepaired instanceof LeveledCompactionStrategy)
 +        {
 +            int count = 0;
 +            count += ((LeveledCompactionStrategy)repaired).getLevelSize(0);
 +            count += ((LeveledCompactionStrategy)unrepaired).getLevelSize(0);
 +            return count;
 +        }
 +        return 0;
 +    }
 +
 +    public synchronized int[] getSSTableCountPerLevel()
 +    {
 +        if (repaired instanceof LeveledCompactionStrategy && unrepaired instanceof LeveledCompactionStrategy)
 +        {
 +            int [] res = new int[LeveledManifest.MAX_LEVEL_COUNT];
 +            int[] repairedCountPerLevel = ((LeveledCompactionStrategy) repaired).getAllLevelSize();
 +            res = sumArrays(res, repairedCountPerLevel);
 +            int[] unrepairedCountPerLevel = ((LeveledCompactionStrategy) unrepaired).getAllLevelSize();
 +            res = sumArrays(res, unrepairedCountPerLevel);
 +            return res;
 +        }
 +        return null;
 +    }
 +
 +    private static int[] sumArrays(int[] a, int[] b)
 +    {
 +        int[] res = new int[Math.max(a.length, b.length)];
 +        for (int i = 0; i < res.length; i++)
 +        {
 +            if (i < a.length && i < b.length)
 +                res[i] = a[i] + b[i];
 +            else if (i < a.length)
 +                res[i] = a[i];
 +            else
 +                res[i] = b[i];
 +        }
 +        return res;
 +    }
 +
 +    public boolean shouldDefragment()
 +    {
 +        assert repaired.getClass().equals(unrepaired.getClass());
 +        return repaired.shouldDefragment();
 +    }
 +
 +    public Directories getDirectories()
 +    {
 +        assert repaired.getClass().equals(unrepaired.getClass());
 +        return repaired.getDirectories();
 +    }
 +
 +    public synchronized void handleNotification(INotification notification, Object sender)
 +    {
 +        if (notification instanceof SSTableAddedNotification)
 +        {
 +            SSTableAddedNotification flushedNotification = (SSTableAddedNotification) notification;
 +            for (SSTableReader sstable : flushedNotification.added)
 +            {
 +                if (sstable.isRepaired())
 +                    repaired.addSSTable(sstable);
 +                else
 +                    unrepaired.addSSTable(sstable);
 +            }
 +        }
 +        else if (notification instanceof SSTableListChangedNotification)
 +        {
 +            SSTableListChangedNotification listChangedNotification = (SSTableListChangedNotification) notification;
 +            Set<SSTableReader> repairedRemoved = new HashSet<>();
 +            Set<SSTableReader> repairedAdded = new HashSet<>();
 +            Set<SSTableReader> unrepairedRemoved = new HashSet<>();
 +            Set<SSTableReader> unrepairedAdded = new HashSet<>();
 +
 +            for (SSTableReader sstable : listChangedNotification.removed)
 +            {
 +                if (sstable.isRepaired())
 +                    repairedRemoved.add(sstable);
 +                else
 +                    unrepairedRemoved.add(sstable);
 +            }
 +            for (SSTableReader sstable : listChangedNotification.added)
 +            {
 +                if (sstable.isRepaired())
 +                    repairedAdded.add(sstable);
 +                else
 +                    unrepairedAdded.add(sstable);
 +            }
 +            if (!repairedRemoved.isEmpty())
 +            {
 +                repaired.replaceSSTables(repairedRemoved, repairedAdded);
 +            }
 +            else
 +            {
 +                for (SSTableReader sstable : repairedAdded)
 +                    repaired.addSSTable(sstable);
 +            }
 +
 +            if (!unrepairedRemoved.isEmpty())
 +            {
 +                unrepaired.replaceSSTables(unrepairedRemoved, unrepairedAdded);
 +            }
 +            else
 +            {
 +                for (SSTableReader sstable : unrepairedAdded)
 +                    unrepaired.addSSTable(sstable);
 +            }
 +        }
 +        else if (notification instanceof SSTableRepairStatusChanged)
 +        {
 +            for (SSTableReader sstable : ((SSTableRepairStatusChanged) notification).sstable)
 +            {
 +                if (sstable.isRepaired())
 +                {
 +                    unrepaired.removeSSTable(sstable);
 +                    repaired.addSSTable(sstable);
 +                }
 +                else
 +                {
 +                    repaired.removeSSTable(sstable);
 +                    unrepaired.addSSTable(sstable);
 +                }
 +            }
 +        }
 +        else if (notification instanceof SSTableDeletingNotification)
 +        {
 +            SSTableReader sstable = ((SSTableDeletingNotification)notification).deleting;
 +            if (sstable.isRepaired())
 +                repaired.removeSSTable(sstable);
 +            else
 +                unrepaired.removeSSTable(sstable);
 +        }
 +    }
 +
 +    public void enable()
 +    {
 +        if (repaired != null)
 +            repaired.enable();
 +        if (unrepaired != null)
 +            unrepaired.enable();
 +        // enable this last to make sure the strategies are ready to get calls.
 +        enabled = true;
 +    }
 +
 +    public void disable()
 +    {
 +        // disable this first avoid asking disabled strategies for compaction tasks
 +        enabled = false;
 +        if (repaired != null)
 +            repaired.disable();
 +        if (unrepaired != null)
 +            unrepaired.disable();
 +    }
 +
 +    /**
 +     * Create ISSTableScanner from the given sstables
 +     *
 +     * Delegates the call to the compaction strategies to allow LCS to create a scanner
 +     * @param sstables
 +     * @param range
 +     * @return
 +     */
 +    @SuppressWarnings("resource")
 +    public synchronized AbstractCompactionStrategy.ScannerList getScanners(Collection<SSTableReader> sstables,  Collection<Range<Token>> ranges)
 +    {
 +        List<SSTableReader> repairedSSTables = new ArrayList<>();
 +        List<SSTableReader> unrepairedSSTables = new ArrayList<>();
 +        for (SSTableReader sstable : sstables)
 +        {
 +            if (sstable.isRepaired())
 +                repairedSSTables.add(sstable);
 +            else
 +                unrepairedSSTables.add(sstable);
 +        }
 +
 +        Set<ISSTableScanner> scanners = new HashSet<>(sstables.size());
 +
 +        for (Range<Token> range : ranges)
 +        {
 +            AbstractCompactionStrategy.ScannerList repairedScanners = repaired.getScanners(repairedSSTables, range);
 +            AbstractCompactionStrategy.ScannerList unrepairedScanners = unrepaired.getScanners(unrepairedSSTables, range);
 +
 +            for (ISSTableScanner scanner : Iterables.concat(repairedScanners.scanners, unrepairedScanners.scanners))
 +            {
 +                if (!scanners.add(scanner))
 +                    scanner.close();
 +            }
 +        }
 +
 +        return new AbstractCompactionStrategy.ScannerList(new ArrayList<>(scanners));
 +    }
 +
 +    public synchronized AbstractCompactionStrategy.ScannerList getScanners(Collection<SSTableReader> sstables)
 +    {
 +        return getScanners(sstables, Collections.singleton(null));
 +    }
 +
 +    public Collection<Collection<SSTableReader>> groupSSTablesForAntiCompaction(Collection<SSTableReader> sstablesToGroup)
 +    {
 +        return unrepaired.groupSSTablesForAntiCompaction(sstablesToGroup);
 +    }
 +
 +    public long getMaxSSTableBytes()
 +    {
 +        return unrepaired.getMaxSSTableBytes();
 +    }
 +
 +    public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes)
 +    {
 +        return getCompactionStrategyFor(txn.originals().iterator().next()).getCompactionTask(txn, gcBefore, maxSSTableBytes);
 +    }
 +
 +    public Collection<AbstractCompactionTask> getMaximalTasks(final int gcBefore, final boolean splitOutput)
 +    {
 +        // runWithCompactionsDisabled cancels active compactions and disables them, then we are able
 +        // to make the repaired/unrepaired strategies mark their own sstables as compacting. Once the
 +        // sstables are marked the compactions are re-enabled
 +        return cfs.runWithCompactionsDisabled(new Callable<Collection<AbstractCompactionTask>>()
 +        {
 +            @Override
 +            public Collection<AbstractCompactionTask> call() throws Exception
 +            {
 +                synchronized (CompactionStrategyManager.this)
 +                {
 +                    Collection<AbstractCompactionTask> repairedTasks = repaired.getMaximalTask(gcBefore, splitOutput);
 +                    Collection<AbstractCompactionTask> unrepairedTasks = unrepaired.getMaximalTask(gcBefore, splitOutput);
 +
 +                    if (repairedTasks == null && unrepairedTasks == null)
 +                        return null;
 +
 +                    if (repairedTasks == null)
 +                        return unrepairedTasks;
 +                    if (unrepairedTasks == null)
 +                        return repairedTasks;
 +
 +                    List<AbstractCompactionTask> tasks = new ArrayList<>();
 +                    tasks.addAll(repairedTasks);
 +                    tasks.addAll(unrepairedTasks);
 +                    return tasks;
 +                }
 +            }
 +        }, false, false);
 +    }
 +
 +    public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, int gcBefore)
 +    {
 +        return getCompactionStrategyFor(sstables.iterator().next()).getUserDefinedTask(sstables, gcBefore);
 +    }
 +
 +    public int getEstimatedRemainingTasks()
 +    {
 +        int tasks = 0;
 +        tasks += repaired.getEstimatedRemainingTasks();
 +        tasks += unrepaired.getEstimatedRemainingTasks();
 +
 +        return tasks;
 +    }
 +
 +    public boolean shouldBeEnabled()
 +    {
 +        return params.isEnabled();
 +    }
 +
 +    public String getName()
 +    {
 +        return unrepaired.getName();
 +    }
 +
 +    public List<AbstractCompactionStrategy> getStrategies()
 +    {
 +        return Arrays.asList(repaired, unrepaired);
 +    }
 +
 +    public synchronized void setNewLocalCompactionStrategy(CompactionParams params)
 +    {
 +        logger.info("Switching local compaction strategy from {} to {}}", this.params, params);
 +        setStrategy(params);
 +        if (shouldBeEnabled())
 +            enable();
 +        else
 +            disable();
 +        startup();
 +    }
 +
 +    private void setStrategy(CompactionParams params)
 +    {
 +        if (repaired != null)
 +            repaired.shutdown();
 +        if (unrepaired != null)
 +            unrepaired.shutdown();
 +        repaired = CFMetaData.createCompactionStrategyInstance(cfs, params);
 +        unrepaired = CFMetaData.createCompactionStrategyInstance(cfs, params);
 +        this.params = params;
 +    }
 +
 +    public CompactionParams getCompactionParams()
 +    {
 +        return params;
 +    }
 +
 +    public boolean onlyPurgeRepairedTombstones()
 +    {
 +        return Boolean.parseBoolean(params.options().get(AbstractCompactionStrategy.ONLY_PURGE_REPAIRED_TOMBSTONES));
 +    }
 +
 +    public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, MetadataCollector collector, SerializationHeader header, LifecycleTransaction txn)
 +    {
 +        if (repairedAt == ActiveRepairService.UNREPAIRED_SSTABLE)
 +        {
 +            return unrepaired.createSSTableMultiWriter(descriptor, keyCount, repairedAt, collector, header, txn);
 +        }
 +        else
 +        {
 +            return repaired.createSSTableMultiWriter(descriptor, keyCount, repairedAt, collector, header, txn);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 1d96324,575c326..be81c80
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@@ -142,11 -138,13 +142,11 @@@ public class CompactionTask extends Abs
              ssTableLoggerMsg.append(String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel()));
          }
          ssTableLoggerMsg.append("]");
 -        String taskIdLoggerMsg = taskId == null ? UUIDGen.getTimeUUID().toString() : taskId.toString();
 -        logger.debug("Compacting ({}) {}", taskIdLoggerMsg, ssTableLoggerMsg);
  
-         logger.info("Compacting ({}) {}", taskId, ssTableLoggerMsg);
 -        long start = System.nanoTime();
++        logger.debug("Compacting ({}) {}", taskId, ssTableLoggerMsg);
  
 +        long start = System.nanoTime();
          long totalKeysWritten = 0;
 -
          long estimatedKeys = 0;
          try (CompactionController controller = getCompactionController(transaction.originals()))
          {
@@@ -213,11 -220,11 +213,11 @@@
  
              double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
              long totalSourceRows = 0;
 -            String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
 +            String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
-             logger.info(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
+             logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
 -                                      taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
 +                                      taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
-             logger.debug(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-             logger.debug("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
+             logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
+             logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
  
              if (offline)
                  Refs.release(Refs.selfRefs(newSStables));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index 7fd5717,d90318f..b7bf83f
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@@ -338,9 -338,9 +338,9 @@@ public class LeveledManifes
                  {
                      int nextLevel = getNextLevel(candidates);
                      candidates = getOverlappingStarvedSSTables(nextLevel, candidates);
-                     if (logger.isDebugEnabled())
-                         logger.debug("Compaction candidates for L{} are {}", i, toString(candidates));
+                     if (logger.isTraceEnabled())
+                         logger.trace("Compaction candidates for L{} are {}", i, toString(candidates));
 -                    return new CompactionCandidate(candidates, nextLevel, cfs.getCompactionStrategy().getMaxSSTableBytes());
 +                    return new CompactionCandidate(candidates, nextLevel, cfs.getCompactionStrategyManager().getMaxSSTableBytes());
                  }
                  else
                  {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index 05f446c,b4125bb..f8a8240
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@@ -80,10 -79,10 +80,10 @@@ public class SizeTieredCompactionStrate
          int minThreshold = cfs.getMinimumCompactionThreshold();
          int maxThreshold = cfs.getMaximumCompactionThreshold();
  
 -        Iterable<SSTableReader> candidates = filterSuspectSSTables(Sets.intersection(cfs.getUncompactingSSTables(), sstables));
 +        Iterable<SSTableReader> candidates = filterSuspectSSTables(filter(cfs.getUncompactingSSTables(), sstables::contains));
  
          List<List<SSTableReader>> buckets = getBuckets(createSSTableAndLengthPairs(candidates), sizeTieredOptions.bucketHigh, sizeTieredOptions.bucketLow, sizeTieredOptions.minSSTableSize);
-         logger.debug("Compaction buckets are {}", buckets);
+         logger.trace("Compaction buckets are {}", buckets);
          updateEstimatedCompactionsByTasks(buckets);
          List<SSTableReader> mostInteresting = mostInterestingBucket(buckets, minThreshold, maxThreshold);
          if (!mostInteresting.isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
index 07ca3d0,ed07df9..796391c
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@@ -82,10 -81,19 +82,10 @@@ public class SplittingSizeTieredCompact
              }
          }
          ratios = Arrays.copyOfRange(potentialRatios, 0, noPointIndex);
 -        File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(Math.round(totalSize * ratios[currentRatioIndex])));
          long currentPartitionsToWrite = Math.round(estimatedTotalKeys * ratios[currentRatioIndex]);
          currentBytesToWrite = Math.round(totalSize * ratios[currentRatioIndex]);
 -        @SuppressWarnings("resource")
 -        SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
 -                                                                            currentPartitionsToWrite,
 -                                                                            minRepairedAt,
 -                                                                            cfs.metadata,
 -                                                                            cfs.partitioner,
 -                                                                            new MetadataCollector(allSSTables, cfs.metadata.comparator, 0));
 -
 -        sstableWriter.switchWriter(writer);
 +        switchCompactionLocation(getWriteDirectory(currentBytesToWrite));
-         logger.debug("Ratios={}, expectedKeys = {}, totalSize = {}, currentPartitionsToWrite = {}, currentBytesToWrite = {}", ratios, estimatedTotalKeys, totalSize, currentPartitionsToWrite, currentBytesToWrite);
+         logger.trace("Ratios={}, expectedKeys = {}, totalSize = {}, currentPartitionsToWrite = {}, currentBytesToWrite = {}", ratios, estimatedTotalKeys, totalSize, currentPartitionsToWrite, currentBytesToWrite);
      }
  
      @Override
@@@ -100,20 -118,4 +100,20 @@@
          }
          return rie != null;
      }
 -}
 +
 +    public void switchCompactionLocation(Directories.DataDirectory location)
 +    {
 +        long currentPartitionsToWrite = Math.round(ratios[currentRatioIndex] * estimatedTotalKeys);
 +        @SuppressWarnings("resource")
 +        SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(getDirectories().getLocationForDisk(location))),
 +                                                    currentPartitionsToWrite,
 +                                                    minRepairedAt,
 +                                                    cfs.metadata,
 +                                                    new MetadataCollector(allSSTables, cfs.metadata.comparator, 0),
 +                                                    SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
 +                                                    txn);
-         logger.debug("Switching writer, currentPartitionsToWrite = {}", currentPartitionsToWrite);
++        logger.trace("Switching writer, currentPartitionsToWrite = {}", currentPartitionsToWrite);
 +        sstableWriter.switchWriter(writer);
 +
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
index 59bbc7d,9b52269..83d0f82
--- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@@ -222,14 -149,9 +222,14 @@@ public class LifecycleTransaction exten
      public Throwable doCommit(Throwable accumulate)
      {
          assert staged.isEmpty() : "must be no actions introduced between prepareToCommit and a commit";
-         logger.debug("Committing update:{}, obsolete:{}", staged.update, staged.obsolete);
 -
+         logger.trace("Committing update:{}, obsolete:{}", staged.update, staged.obsolete);
  
 +        // accumulate must be null if we have been used correctly, so fail immediately if it is not
 +        maybeFail(accumulate);
 +
 +        // transaction log commit failure means we must abort; safe commit is not possible
 +        maybeFail(log.commit(null));
 +
          // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done
          // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size
          // and notification status for the obsolete and new files
@@@ -247,23 -167,18 +247,23 @@@
       */
      public Throwable doAbort(Throwable accumulate)
      {
-         if (logger.isDebugEnabled())
-             logger.debug("Aborting transaction over {}, with ({},{}) logged and ({},{}) staged", originals, logged.update, logged.obsolete, staged.update, staged.obsolete);
+         if (logger.isTraceEnabled())
+             logger.trace("Aborting transaction over {}, with ({},{}) logged and ({},{}) staged", originals, logged.update, logged.obsolete, staged.update, staged.obsolete);
  
 +        accumulate = abortObsoletion(obsoletions, accumulate);
 +
          if (logged.isEmpty() && staged.isEmpty())
 -            return accumulate;
 +            return log.abort(accumulate);
  
          // mark obsolete all readers that are not versions of those present in the original set
          Iterable<SSTableReader> obsolete = filterOut(concatUniq(staged.update, logged.update), originals);
-         logger.debug("Obsoleting {}", obsolete);
+         logger.trace("Obsoleting {}", obsolete);
 -        // we don't pass the tracker in for the obsoletion, since these readers have never been notified externally
 -        // nor had their size accounting affected
 -        accumulate = markObsolete(null, obsolete, accumulate);
 +
 +        accumulate = prepareForObsoletion(obsolete, log, obsoletions = new ArrayList<>(), accumulate);
 +        // it's safe to abort even if committed, see maybeFail in doCommit() above, in this case it will just report
 +        // a failure to abort, which is useful information to have for debug
 +        accumulate = log.abort(accumulate);
 +        accumulate = markObsolete(obsoletions, accumulate);
  
          // replace all updated readers with a version restored to its original state
          accumulate = tracker.apply(updateLiveSet(logged.update, restoreUpdatedOriginals()), accumulate);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
index da6d78d,0000000..78ea0f1
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@@ -1,419 -1,0 +1,419 @@@
 +/*
 + * 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.db.lifecycle;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.nio.file.Files;
 +import java.nio.file.NoSuchFileException;
 +import java.util.*;
 +import java.util.concurrent.ConcurrentLinkedQueue;
 +import java.util.concurrent.TimeUnit;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.util.concurrent.Runnables;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.compaction.OperationType;
 +import org.apache.cassandra.db.lifecycle.LogRecord.Type;
 +import org.apache.cassandra.io.sstable.Component;
 +import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.Ref;
 +import org.apache.cassandra.utils.concurrent.RefCounted;
 +import org.apache.cassandra.utils.concurrent.Transactional;
 +
 +/**
 + * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
 + * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
 + * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
 + * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
 + *
 + * IMPORTANT: The transaction must complete (commit or abort) before any temporary files are deleted, even though the
 + * txn log file itself will not be deleted until all tracked files are deleted. This is required by FileLister to ensure
 + * a consistent disk state. LifecycleTransaction ensures this requirement, so this class should really never be used
 + * outside of LT. @see FileLister.classifyFiles(TransactionData txn)
 + *
 + * A class that tracks sstable files involved in a transaction across sstables:
 + * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
 + *
 + * The transaction log file contains new and old sstables as follows:
 + *
 + * add:[sstable-2][CRC]
 + * remove:[sstable-1,max_update_time,num files][CRC]
 + *
 + * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
 + * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
 + * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
 + * and file sizes.
 + *
 + * Upon commit we add a final line to the log file:
 + *
 + * commit:[commit_time][CRC]
 + *
 + * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
 + * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
 + * was committed, vice-versa if the transaction was aborted.
 + *
 + * On start-up we look for any transaction log files and repeat the cleanup process described above.
 + *
 + * See CASSANDRA-7066 for full details.
 + */
 +class LogTransaction extends Transactional.AbstractTransactional implements Transactional
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LogTransaction.class);
 +
 +    /**
 +     * If the format of the lines in the transaction log is wrong or the checksum
 +     * does not match, then we throw this exception.
 +     */
 +    public static final class CorruptTransactionLogException extends RuntimeException
 +    {
 +        public final LogFile file;
 +
 +        public CorruptTransactionLogException(String message, LogFile file)
 +        {
 +            super(message);
 +            this.file = file;
 +        }
 +    }
 +
 +    private final Tracker tracker;
 +    private final LogFile data;
 +    private final Ref<LogTransaction> selfRef;
 +    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
 +    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
 +    // Additionally, we need to make sure to delete the data file first, so on restart the others
 +    // will be recognized as GCable.
 +    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
 +
 +    LogTransaction(OperationType opType, CFMetaData metadata)
 +    {
 +        this(opType, metadata, null);
 +    }
 +
 +    LogTransaction(OperationType opType, CFMetaData metadata, Tracker tracker)
 +    {
 +        this(opType, new Directories(metadata), tracker);
 +    }
 +
 +    LogTransaction(OperationType opType, Directories directories, Tracker tracker)
 +    {
 +        this(opType, directories.getDirectoryForNewSSTables(), tracker);
 +    }
 +
 +    LogTransaction(OperationType opType, File folder, Tracker tracker)
 +    {
 +        this.tracker = tracker;
 +        int folderDescriptor = CLibrary.tryOpenDirectory(folder.getPath());
 +        this.data = new LogFile(opType, folder, folderDescriptor, UUIDGen.getTimeUUID());
 +        this.selfRef = new Ref<>(this, new TransactionTidier(data, folderDescriptor));
 +
-         if (logger.isDebugEnabled())
-             logger.debug("Created transaction logs with id {}", data.id);
++        if (logger.isTraceEnabled())
++            logger.trace("Created transaction logs with id {}", data.id);
 +    }
 +
 +    /**
 +     * Track a reader as new.
 +     **/
 +    void trackNew(SSTable table)
 +    {
 +        data.add(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Stop tracking a reader as new.
 +     */
 +    void untrackNew(SSTable table)
 +    {
 +        data.remove(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Schedule a reader for deletion as soon as it is fully unreferenced.
 +     */
 +    SSTableTidier obsoleted(SSTableReader reader)
 +    {
 +        if (data.contains(Type.ADD, reader))
 +        {
 +            if (data.contains(Type.REMOVE, reader))
 +                throw new IllegalArgumentException();
 +
 +            return new SSTableTidier(reader, true, this);
 +        }
 +
 +        data.add(Type.REMOVE, reader);
 +
 +        if (tracker != null)
 +            tracker.notifyDeleting(reader);
 +
 +        return new SSTableTidier(reader, false, this);
 +    }
 +
 +    OperationType getType()
 +    {
 +        return data.getType();
 +    }
 +
 +    UUID getId()
 +    {
 +        return data.getId();
 +    }
 +
 +    @VisibleForTesting
 +    String getDataFolder()
 +    {
 +        return data.folder.getPath();
 +    }
 +
 +    @VisibleForTesting
 +    LogFile getLogFile()
 +    {
 +        return data;
 +    }
 +
 +    static void delete(File file)
 +    {
 +        try
 +        {
-             if (logger.isDebugEnabled())
-                 logger.debug("Deleting {}", file);
++            if (logger.isTraceEnabled())
++                logger.trace("Deleting {}", file);
 +
 +            Files.delete(file.toPath());
 +        }
 +        catch (NoSuchFileException e)
 +        {
 +            logger.error("Unable to delete {} as it does not exist", file);
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Unable to delete {}", file, e);
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    /**
 +     * The transaction tidier.
 +     *
 +     * When the transaction reference is fully released we try to delete all the obsolete files
 +     * depending on the transaction result, as well as the transaction log file.
 +     */
 +    private static class TransactionTidier implements RefCounted.Tidy, Runnable
 +    {
 +        private final LogFile data;
 +        private final int folderDescriptor;
 +
 +        TransactionTidier(LogFile data, int folderDescriptor)
 +        {
 +            this.data = data;
 +            this.folderDescriptor = folderDescriptor;
 +        }
 +
 +        public void tidy() throws Exception
 +        {
 +            run();
 +        }
 +
 +        public String name()
 +        {
 +            return data.toString();
 +        }
 +
 +        public void run()
 +        {
-             if (logger.isDebugEnabled())
-                 logger.debug("Removing files for transaction {}", name());
++            if (logger.isTraceEnabled())
++                logger.trace("Removing files for transaction {}", name());
 +
 +            assert data.completed() : "Expected a completed transaction: " + data;
 +
 +            Throwable err = data.removeUnfinishedLeftovers(null);
 +
 +            if (err != null)
 +            {
 +                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
 +                failedDeletions.add(this);
 +            }
 +            else
 +            {
-                 if (logger.isDebugEnabled())
-                     logger.debug("Closing file transaction {}", name());
++                if (logger.isTraceEnabled())
++                    logger.trace("Closing file transaction {}", name());
 +                CLibrary.tryCloseFD(folderDescriptor);
 +            }
 +        }
 +    }
 +
 +    static class Obsoletion
 +    {
 +        final SSTableReader reader;
 +        final SSTableTidier tidier;
 +
 +        Obsoletion(SSTableReader reader, SSTableTidier tidier)
 +        {
 +            this.reader = reader;
 +            this.tidier = tidier;
 +        }
 +    }
 +
 +    /**
 +     * The SSTableReader tidier. When a reader is fully released and no longer referenced
 +     * by any one, we run this. It keeps a reference to the parent transaction and releases
 +     * it when done, so that the final transaction cleanup can run when all obsolete readers
 +     * are released.
 +     */
 +    public static class SSTableTidier implements Runnable
 +    {
 +        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
 +        private final Descriptor desc;
 +        private final long sizeOnDisk;
 +        private final Tracker tracker;
 +        private final boolean wasNew;
 +        private final Ref<LogTransaction> parentRef;
 +
 +        public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent)
 +        {
 +            this.desc = referent.descriptor;
 +            this.sizeOnDisk = referent.bytesOnDisk();
 +            this.tracker = parent.tracker;
 +            this.wasNew = wasNew;
 +            this.parentRef = parent.selfRef.tryRef();
 +        }
 +
 +        public void run()
 +        {
 +            SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
 +
 +            try
 +            {
 +                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
 +                File datafile = new File(desc.filenameFor(Component.DATA));
 +
 +                delete(datafile);
 +                // let the remainder be cleaned up by delete
 +                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
 +            }
 +            catch (Throwable t)
 +            {
 +                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
 +                failedDeletions.add(this);
 +                return;
 +            }
 +
 +            if (tracker != null && tracker.cfstore != null && !wasNew)
 +                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
 +
 +            // release the referent to the parent so that the all transaction files can be released
 +            parentRef.release();
 +        }
 +
 +        public void abort()
 +        {
 +            parentRef.release();
 +        }
 +    }
 +
 +
 +    static void rescheduleFailedDeletions()
 +    {
 +        Runnable task;
 +        while ( null != (task = failedDeletions.poll()))
 +            ScheduledExecutors.nonPeriodicTasks.submit(task);
 +
 +        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
 +        SnapshotDeletingTask.rescheduleFailedTasks();
 +    }
 +
 +    static void waitForDeletions()
 +    {
 +        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
 +    }
 +
 +    @VisibleForTesting
 +    Throwable complete(Throwable accumulate)
 +    {
 +        try
 +        {
 +            accumulate = selfRef.ensureReleased(accumulate);
 +            return accumulate;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.error("Failed to complete file transaction {}", getId(), t);
 +            return Throwables.merge(accumulate, t);
 +        }
 +    }
 +
 +    protected Throwable doCommit(Throwable accumulate)
 +    {
 +        data.commit();
 +        return complete(accumulate);
 +    }
 +
 +    protected Throwable doAbort(Throwable accumulate)
 +    {
 +        data.abort();
 +        return complete(accumulate);
 +    }
 +
 +    protected void doPrepare() { }
 +
 +    /**
 +     * Called on startup to scan existing folders for any unfinished leftovers of
 +     * operations that were ongoing when the process exited. Also called by the standalone
 +     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
 +     *
 +     */
 +    static void removeUnfinishedLeftovers(CFMetaData metadata)
 +    {
 +        for (File dir : new Directories(metadata).getCFDirectories())
 +        {
 +            int folderDescriptor = CLibrary.tryOpenDirectory(dir.getPath());
 +            try
 +            {
 +                File[] logs = dir.listFiles(LogFile::isLogFile);
 +
 +                for (File log : logs)
 +                {
 +                    LogFile data = LogFile.make(log, folderDescriptor);
 +                    data.readRecords();
 +                    if (data.verify())
 +                    {
 +                        Throwable failure = data.removeUnfinishedLeftovers(null);
 +                        if (failure != null)
 +                            logger.error("Failed to remove unfinished transaction leftovers for log {}", log, failure);
 +                    }
 +                    else
 +                    {
 +                        logger.error("Unexpected disk state: failed to read transaction log {}", log);
 +                    }
 +                }
 +            }
 +            finally
 +            {
 +                CLibrary.tryCloseFD(folderDescriptor);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/dht/BootStrapper.java
index 31fda34,a6b1ad7..c0f0402
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@@ -178,34 -181,6 +178,34 @@@ public class BootStrapper extends Progr
          return getRandomTokens(metadata, numTokens);
      }
  
 +    private static Collection<Token> getSpecifiedTokens(final TokenMetadata metadata,
 +                                                        Collection<String> initialTokens)
 +    {
-         logger.debug("tokens manually specified as {}",  initialTokens);
++        logger.trace("tokens manually specified as {}",  initialTokens);
 +        List<Token> tokens = new ArrayList<>(initialTokens.size());
 +        for (String tokenString : initialTokens)
 +        {
 +            Token token = metadata.partitioner.getTokenFactory().fromString(tokenString);
 +            if (metadata.getEndpoint(token) != null)
 +                throw new ConfigurationException("Bootstrapping to existing token " + tokenString + " is not allowed (decommission/removenode the old node first).");
 +            tokens.add(token);
 +        }
 +        return tokens;
 +    }
 +
 +    static Collection<Token> allocateTokens(final TokenMetadata metadata,
 +                                            InetAddress address,
 +                                            String allocationKeyspace,
 +                                            int numTokens)
 +    {
 +        Keyspace ks = Keyspace.open(allocationKeyspace);
 +        if (ks == null)
 +            throw new ConfigurationException("Problem opening token allocation keyspace " + allocationKeyspace);
 +        AbstractReplicationStrategy rs = ks.getReplicationStrategy();
 +
 +        return TokenAllocation.allocateTokens(metadata, rs, address, numTokens);
 +    }
 +
      public static Collection<Token> getRandomTokens(TokenMetadata metadata, int numTokens)
      {
          Set<Token> tokens = new HashSet<>(numTokens);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/dht/RangeStreamer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
index 3f74c33,36da92d..ec5167b
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
@@@ -98,238 -78,4 +98,238 @@@ public class CqlInputFormat extends org
          return new CqlRecordReader();
      }
  
 +    protected void validateConfiguration(Configuration conf)
 +    {
 +        if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null)
 +        {
 +            throw new UnsupportedOperationException("you must set the keyspace and table with setInputColumnFamily()");
 +        }
 +        if (ConfigHelper.getInputInitialAddress(conf) == null)
 +            throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress");
 +        if (ConfigHelper.getInputPartitioner(conf) == null)
 +            throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner");
 +    }
 +
 +    public List<org.apache.hadoop.mapreduce.InputSplit> getSplits(JobContext context) throws IOException
 +    {
 +        Configuration conf = HadoopCompat.getConfiguration(context);
 +
 +        validateConfiguration(conf);
 +
 +        keyspace = ConfigHelper.getInputKeyspace(conf);
 +        cfName = ConfigHelper.getInputColumnFamily(conf);
 +        partitioner = ConfigHelper.getInputPartitioner(conf);
-         logger.debug("partitioner is {}", partitioner);
++        logger.trace("partitioner is {}", partitioner);
 +
 +        // canonical ranges and nodes holding replicas
 +        Map<TokenRange, Set<Host>> masterRangeNodes = getRangeMap(conf, keyspace);
 +
 +        // canonical ranges, split into pieces, fetching the splits in parallel
 +        ExecutorService executor = new ThreadPoolExecutor(0, 128, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
 +        List<org.apache.hadoop.mapreduce.InputSplit> splits = new ArrayList<>();
 +
 +        try
 +        {
 +            List<Future<List<org.apache.hadoop.mapreduce.InputSplit>>> splitfutures = new ArrayList<>();
 +            KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
 +            Range<Token> jobRange = null;
 +            if (jobKeyRange != null)
 +            {
 +                if (jobKeyRange.start_key != null)
 +                {
 +                    if (!partitioner.preservesOrder())
 +                        throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving partitioner");
 +                    if (jobKeyRange.start_token != null)
 +                        throw new IllegalArgumentException("only start_key supported");
 +                    if (jobKeyRange.end_token != null)
 +                        throw new IllegalArgumentException("only start_key supported");
 +                    jobRange = new Range<>(partitioner.getToken(jobKeyRange.start_key),
 +                                           partitioner.getToken(jobKeyRange.end_key));
 +                }
 +                else if (jobKeyRange.start_token != null)
 +                {
 +                    jobRange = new Range<>(partitioner.getTokenFactory().fromString(jobKeyRange.start_token),
 +                                           partitioner.getTokenFactory().fromString(jobKeyRange.end_token));
 +                }
 +                else
 +                {
 +                    logger.warn("ignoring jobKeyRange specified without start_key or start_token");
 +                }
 +            }
 +
 +            session = CqlConfigHelper.getInputCluster(ConfigHelper.getInputInitialAddress(conf).split(","), conf).connect();
 +            Metadata metadata = session.getCluster().getMetadata();
 +
 +            for (TokenRange range : masterRangeNodes.keySet())
 +            {
 +                if (jobRange == null)
 +                {
 +                    // for each tokenRange, pick a live owner and ask it to compute bite-sized splits
 +                    splitfutures.add(executor.submit(new SplitCallable(range, masterRangeNodes.get(range), conf)));
 +                }
 +                else
 +                {
 +                    TokenRange jobTokenRange = rangeToTokenRange(metadata, jobRange);
 +                    if (range.intersects(jobTokenRange))
 +                    {
 +                        for (TokenRange intersection: range.intersectWith(jobTokenRange))
 +                        {
 +                            // for each tokenRange, pick a live owner and ask it to compute bite-sized splits
 +                            splitfutures.add(executor.submit(new SplitCallable(intersection,  masterRangeNodes.get(range), conf)));
 +                        }
 +                    }
 +                }
 +            }
 +
 +            // wait until we have all the results back
 +            for (Future<List<org.apache.hadoop.mapreduce.InputSplit>> futureInputSplits : splitfutures)
 +            {
 +                try
 +                {
 +                    splits.addAll(futureInputSplits.get());
 +                }
 +                catch (Exception e)
 +                {
 +                    throw new IOException("Could not get input splits", e);
 +                }
 +            }
 +        }
 +        finally
 +        {
 +            executor.shutdownNow();
 +        }
 +
 +        assert splits.size() > 0;
 +        Collections.shuffle(splits, new Random(System.nanoTime()));
 +        return splits;
 +    }
 +
 +    private TokenRange rangeToTokenRange(Metadata metadata, Range<Token> range)
 +    {
 +        return metadata.newTokenRange(metadata.newToken(partitioner.getTokenFactory().toString(range.left)),
 +                metadata.newToken(partitioner.getTokenFactory().toString(range.right)));
 +    }
 +
 +    private Map<TokenRange, Long> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException
 +    {
 +        int splitSize = ConfigHelper.getInputSplitSize(conf);
 +        try
 +        {
 +            return describeSplits(keyspace, cfName, range, splitSize);
 +        }
 +        catch (Exception e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    private Map<TokenRange, Set<Host>> getRangeMap(Configuration conf, String keyspace)
 +    {
 +        try (Session session = CqlConfigHelper.getInputCluster(ConfigHelper.getInputInitialAddress(conf).split(","), conf).connect())
 +        {
 +            Map<TokenRange, Set<Host>> map = new HashMap<>();
 +            Metadata metadata = session.getCluster().getMetadata();
 +            for (TokenRange tokenRange : metadata.getTokenRanges())
 +                map.put(tokenRange, metadata.getReplicas('"' + keyspace + '"', tokenRange));
 +            return map;
 +        }
 +    }
 +
 +    private Map<TokenRange, Long> describeSplits(String keyspace, String table, TokenRange tokenRange, int splitSize)
 +    {
 +        String query = String.format("SELECT mean_partition_size, partitions_count " +
 +                                     "FROM %s.%s " +
 +                                     "WHERE keyspace_name = ? AND table_name = ? AND range_start = ? AND range_end = ?",
 +                                     SystemKeyspace.NAME,
 +                                     SystemKeyspace.SIZE_ESTIMATES);
 +
 +        ResultSet resultSet = session.execute(query, keyspace, table, tokenRange.getStart().toString(), tokenRange.getEnd().toString());
 +
 +        Row row = resultSet.one();
 +        // If we have no data on this split, return the full split i.e., do not sub-split
 +        // Assume smallest granularity of partition count available from CASSANDRA-7688
 +        if (row == null)
 +        {
 +            Map<TokenRange, Long> wrappedTokenRange = new HashMap<>();
 +            wrappedTokenRange.put(tokenRange, (long) 128);
 +            return wrappedTokenRange;
 +        }
 +
 +        long meanPartitionSize = row.getLong("mean_partition_size");
 +        long partitionCount = row.getLong("partitions_count");
 +
 +        int splitCount = (int)((meanPartitionSize * partitionCount) / splitSize);
 +        List<TokenRange> splitRanges = tokenRange.splitEvenly(splitCount);
 +        Map<TokenRange, Long> rangesWithLength = new HashMap<>();
 +        for (TokenRange range : splitRanges)
 +            rangesWithLength.put(range, partitionCount/splitCount);
 +
 +        return rangesWithLength;
 +    }
 +
 +    // Old Hadoop API
 +    public InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException
 +    {
 +        TaskAttemptContext tac = HadoopCompat.newTaskAttemptContext(jobConf, new TaskAttemptID());
 +        List<org.apache.hadoop.mapreduce.InputSplit> newInputSplits = this.getSplits(tac);
 +        InputSplit[] oldInputSplits = new InputSplit[newInputSplits.size()];
 +        for (int i = 0; i < newInputSplits.size(); i++)
 +            oldInputSplits[i] = (ColumnFamilySplit)newInputSplits.get(i);
 +        return oldInputSplits;
 +    }
 +
 +    /**
 +     * Gets a token tokenRange and splits it up according to the suggested
 +     * size into input splits that Hadoop can use.
 +     */
 +    class SplitCallable implements Callable<List<org.apache.hadoop.mapreduce.InputSplit>>
 +    {
 +
 +        private final TokenRange tokenRange;
 +        private final Set<Host> hosts;
 +        private final Configuration conf;
 +
 +        public SplitCallable(TokenRange tr, Set<Host> hosts, Configuration conf)
 +        {
 +            this.tokenRange = tr;
 +            this.hosts = hosts;
 +            this.conf = conf;
 +        }
 +
 +        public List<org.apache.hadoop.mapreduce.InputSplit> call() throws Exception
 +        {
 +            ArrayList<org.apache.hadoop.mapreduce.InputSplit> splits = new ArrayList<>();
 +            Map<TokenRange, Long> subSplits;
 +            subSplits = getSubSplits(keyspace, cfName, tokenRange, conf);
 +            // turn the sub-ranges into InputSplits
 +            String[] endpoints = new String[hosts.size()];
 +
 +            // hadoop needs hostname, not ip
 +            int endpointIndex = 0;
 +            for (Host endpoint : hosts)
 +                endpoints[endpointIndex++] = endpoint.getAddress().getHostName();
 +
 +            boolean partitionerIsOpp = partitioner instanceof OrderPreservingPartitioner || partitioner instanceof ByteOrderedPartitioner;
 +
 +            for (TokenRange subSplit : subSplits.keySet())
 +            {
 +                List<TokenRange> ranges = subSplit.unwrap();
 +                for (TokenRange subrange : ranges)
 +                {
 +                    ColumnFamilySplit split =
 +                            new ColumnFamilySplit(
 +                                    partitionerIsOpp ?
 +                                            subrange.getStart().toString().substring(2) : subrange.getStart().toString(),
 +                                    partitionerIsOpp ?
 +                                            subrange.getEnd().toString().substring(2) : subrange.getStart().toString(),
 +                                    subSplits.get(subSplit),
 +                                    endpoints);
 +
-                     logger.debug("adding {}", split);
++                    logger.trace("adding {}", split);
 +                    splits.add(split);
 +                }
 +            }
 +            return splits;
 +        }
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/aa60cde3/src/java/org/apache/cassandra/hints/HintVerbHandler.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/hints/HintVerbHandler.java
index 458d01f,0000000..b2c7b6a
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/hints/HintVerbHandler.java
+++ b/src/java/org/apache/cassandra/hints/HintVerbHandler.java
@@@ -1,89 -1,0 +1,89 @@@
 +/*
 + * 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.hints;
 +
 +import java.net.InetAddress;
 +import java.util.UUID;
 +
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.net.IVerbHandler;
 +import org.apache.cassandra.net.MessageIn;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.service.StorageService;
 +
 +/**
 + * Verb handler used both for hint dispatch and streaming.
 + *
 + * With the non-sstable format, we cannot just stream hint sstables on node decommission. So sometimes, at decommission
 + * time, we might have to stream hints to a non-owning host (say, if the owning host B is down during decommission of host A).
 + * In that case the handler just stores the received hint in its local hint store.
 + */
 +public final class HintVerbHandler implements IVerbHandler<HintMessage>
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(HintVerbHandler.class);
 +
 +    public void doVerb(MessageIn<HintMessage> message, int id)
 +    {
 +        UUID hostId = message.payload.hostId;
 +        Hint hint = message.payload.hint;
 +
 +        // If we see an unknown table id, it means the table, or one of the tables in the mutation, had been dropped.
 +        // In that case there is nothing we can really do, or should do, other than log it go on.
 +        // This will *not* happen due to a not-yet-seen table, because we don't transfer hints unless there
 +        // is schema agreement between the sender and the receiver.
 +        if (hint == null)
 +        {
-             logger.debug("Failed to decode and apply a hint for {} - table with id {} is unknown",
++            logger.trace("Failed to decode and apply a hint for {} - table with id {} is unknown",
 +                         hostId,
 +                         message.payload.unknownTableID);
 +            reply(id, message.from);
 +            return;
 +        }
 +
 +        // We must perform validation before applying the hint, and there is no other place to do it other than here.
 +        try
 +        {
 +            hint.mutation.getPartitionUpdates().forEach(PartitionUpdate::validate);
 +        }
 +        catch (MarshalException e)
 +        {
 +            logger.warn("Failed to validate a hint for {} (table id {}) - skipped", hostId);
 +            reply(id, message.from);
 +            return;
 +        }
 +
 +        // Apply the hint if this node is the destination, store for future dispatch if this node isn't (must have gotten
 +        // it from a decommissioned node that had streamed it before going out).
 +        if (hostId.equals(StorageService.instance.getLocalHostUUID()))
 +            hint.apply();
 +        else
 +            HintsService.instance.write(hostId, hint);
 +
 +        reply(id, message.from);
 +    }
 +
 +    private static void reply(int id, InetAddress to)
 +    {
 +        MessagingService.instance().sendReply(HintResponse.message, id, to);
 +    }
 +}