You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2016/03/29 11:12:47 UTC

[01/15] cassandra git commit: Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 a9b542205 -> 8b8a3f5b9
  refs/heads/cassandra-2.2 6c1ef2ba4 -> 897ffe87e
  refs/heads/cassandra-3.0 3efc609e0 -> 0c91977da
  refs/heads/cassandra-3.5 c7ef7c91c -> 0a2508544
  refs/heads/trunk a0a949440 -> f04224707


Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Patch by marcuse; reviewed by Carl Yeksigian for CASSANDRA-11179


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

Branch: refs/heads/cassandra-2.1
Commit: 8b8a3f5b99fa7a8eefed14cd7e41b81773617046
Parents: a9b5422
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Mar 14 11:01:11 2016 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:50:38 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 12 +++----
 .../db/compaction/CompactionManager.java        | 35 +++++++++++++++-----
 .../cassandra/service/StorageService.java       | 24 +++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 ++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 33 +++++++++++-------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++--
 .../org/apache/cassandra/db/CleanupTest.java    |  6 ++--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 18 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 10 files changed, 111 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5f01114..7794d4f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,6 +17,8 @@
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)
  * Avoid major compaction mixing repaired and unrepaired sstables in DTCS (CASSANDRA-11113)
  * test_bulk_round_trip_blogposts is failing occasionally (CASSANDRA-10938)
+ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
+   many threads to use (CASSANDRA-11179)
 
 
 2.1.13

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 a78f33f..3d66d3a 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1478,22 +1478,22 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return maxFile;
     }
 
-    public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
     }
 
     public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 ec7cb45..e382cab 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -271,7 +271,17 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation) throws ExecutionException, InterruptedException
+    /**
+     * Run an operation over all sstables using jobs threads
+     *
+     * @param cfs the column family store to run the operation on
+     * @param operation the operation to run
+     * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+     * @return status of the operation
+     * @throws ExecutionException
+     * @throws InterruptedException
+     */
+    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
     {
         Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
         if (compactingSSTables == null)
@@ -299,7 +309,8 @@ public class CompactionManager implements CompactionManagerMBean
                     logger.info("Executor has shut down, not submitting task");
                     return AllSSTableOpStatus.ABORTED;
                 }
-                futures.add(executor.submit(new Callable<Object>()
+
+                Callable<Object> callable = new Callable<Object>()
                 {
                     @Override
                     public Object call() throws Exception
@@ -315,7 +326,13 @@ public class CompactionManager implements CompactionManagerMBean
                         }
                         return this;
                     }
-                }));
+                };
+                futures.add(executor.submit(callable));
+                if (jobs > 0 && futures.size() == jobs)
+                {
+                    FBUtilities.waitOnFutures(futures);
+                    futures.clear();
+                }
             }
             FBUtilities.waitOnFutures(futures);
         }
@@ -341,7 +358,7 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
@@ -357,10 +374,10 @@ public class CompactionManager implements CompactionManagerMBean
             {
                 scrubOne(cfs, input, skipCorrupted, checkData);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
     {
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -385,10 +402,10 @@ public class CompactionManager implements CompactionManagerMBean
                 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                 task.execute(metrics);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfStore.isIndex();
         Keyspace keyspace = cfStore.keyspace;
@@ -416,7 +433,7 @@ public class CompactionManager implements CompactionManagerMBean
                 CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
                 doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
             }
-        });
+        }, jobs);
     }
 
     public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 98e2251..507aedb 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2385,13 +2385,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+    }
+
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         if (keyspaceName.equals(Keyspace.SYSTEM_KS))
             throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
 
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup(jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
@@ -2400,27 +2405,36 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, columnFamilies);
+        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
     }
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
         return status.statusCode;
     }
-
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+    }
+
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8fa2433..d3a1725 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -260,7 +260,9 @@ public interface StorageServiceMBean extends NotificationEmitter
     /**
      * Trigger a cleanup of keys on a single keyspace
      */
+    @Deprecated
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@ -270,13 +272,17 @@ public interface StorageServiceMBean extends NotificationEmitter
      */
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
      */
+    @Deprecated
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Flush all memtables for the given column families, or all columnfamilies for the given keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 1ad1147..ab08e9f 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -63,6 +63,7 @@ import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.HintedHandOffManager;
 import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@ -237,42 +238,50 @@ public class NodeProbe implements AutoCloseable
         jmxc.close();
     }
 
-    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
     }
 
-    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
     }
 
-    public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    private void checkJobs(PrintStream out, int jobs)
     {
-        if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+        if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+            out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+    }
+
+    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
+        if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 819049e..1de2e20 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -1138,6 +1138,11 @@ public class NodeTool
         @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
         private List<String> args = new ArrayList<>();
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to cleanup simultaneusly, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1151,7 +1156,7 @@ public class NodeTool
 
                 try
                 {
-                    probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
+                    probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during cleanup", e);
@@ -1267,6 +1272,11 @@ public class NodeTool
                 description = "Do not validate columns using column validator")
         private boolean noValidation = false;
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1277,7 +1287,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -1345,6 +1355,11 @@ public class NodeTool
         @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
         private boolean includeAll = false;
 
+        @Option(title = "jobs",
+                name = {"-j","--jobs"},
+                description = "Number of sstables to upgrade simultaneously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1355,7 +1370,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
+                    probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during enabling auto-compaction", e);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 1d04dfa..7f54ed7 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -83,7 +83,7 @@ public class CleanupTest extends SchemaLoader
         assertEquals(LOOPS, rows.size());
 
         // with one token in the ring, owned by the local node, cleanup should be a no-op
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // ensure max timestamp of the sstables are retained post-cleanup
         assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
@@ -128,7 +128,7 @@ public class CleanupTest extends SchemaLoader
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
 
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // row data should be gone
         rows = Util.getRangeSlice(cfs);
@@ -165,7 +165,7 @@ public class CleanupTest extends SchemaLoader
         tk2[0] = 1;
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         rows = Util.getRangeSlice(cfs);
         assertEquals(0, rows.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 167671b..4efd082 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -99,7 +99,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(1, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -228,7 +228,7 @@ public class ScrubTest extends SchemaLoader
         SSTableReader sstable = cfs.getSSTables().iterator().next();
         overrideWithGarbage(sstable, 0, 2);
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -249,7 +249,7 @@ public class ScrubTest extends SchemaLoader
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
         assert cfs.getSSTables().isEmpty();
     }
 
@@ -268,7 +268,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(10, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -293,7 +293,7 @@ public class ScrubTest extends SchemaLoader
         for (SSTableReader sstable : cfs.getSSTables())
             new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)).delete();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -506,7 +506,7 @@ public class ScrubTest extends SchemaLoader
 
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
         ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@ -516,7 +516,7 @@ public class ScrubTest extends SchemaLoader
         mutation.apply();
         cfs2.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs2, false, false);
+        CompactionManager.instance.performScrub(cfs2, false, false, 2);
     }
 
     /**
@@ -533,7 +533,7 @@ public class ScrubTest extends SchemaLoader
         Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
         mutation.applyUnsafe();
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         assertEquals(1, cfs.getSSTables().size());
     }
@@ -554,7 +554,7 @@ public class ScrubTest extends SchemaLoader
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, true, true);
+        CompactionManager.instance.performScrub(cfs, true, true, 2);
 
         // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
         UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 7d33c11..749056c 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -328,7 +328,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
         cfs.disableAutoCompaction();
-        cfs.sstablesRewrite(false);
+        cfs.sstablesRewrite(false, 2);
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
     }


[12/15] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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

Branch: refs/heads/cassandra-3.0
Commit: 0c91977da68f054cf0ddf996b49423d736b42006
Parents: 3efc609 897ffe8
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 11:00:13 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 11:00:13 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 16 ++++----
 .../db/compaction/CompactionManager.java        | 40 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 23 +++++++++--
 .../cassandra/service/StorageServiceMBean.java  |  6 +++
 .../org/apache/cassandra/tools/NodeProbe.java   | 34 +++++++++++------
 .../cassandra/tools/nodetool/Cleanup.java       |  9 ++++-
 .../apache/cassandra/tools/nodetool/Scrub.java  | 13 +++++--
 .../tools/nodetool/UpgradeSSTable.java          | 10 ++++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 16 ++++----
 11 files changed, 120 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index fe19df7,098d062..fdc873e
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -20,36 -8,13 +20,38 @@@ Merged from 2.2
   * Add missing newline at end of bin/cqlsh (CASSANDRA-11325)
   * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
   * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 - * Fix filtering on non-primary key columns for thrift static column families
 -   (CASSANDRA-6377)
   * Only log yaml config once, at startup (CASSANDRA-11217)
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 +Merged from 2.1:
++ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
++   many threads to use (CASSANDRA-11179)
 + * Backport CASSANDRA-10679 (CASSANDRA-9598)
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
 + * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 +
 +3.0.4
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * MV should only query complex columns included in the view (CASSANDRA-11069)
 + * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
 + * Add sstabledump tool (CASSANDRA-7464)
 + * Introduce backpressure for hints (CASSANDRA-10972)
 + * Fix ClusteringPrefix not being able to read tombstone range boundaries (CASSANDRA-11158)
 + * Prevent logging in sandboxed state (CASSANDRA-11033)
 + * Disallow drop/alter operations of UDTs used by UDAs (CASSANDRA-10721)
 + * Add query time validation method on Index (CASSANDRA-11043)
 + * Avoid potential AssertionError in mixed version cluster (CASSANDRA-11128)
 + * Properly handle hinted handoff after topology changes (CASSANDRA-5902)
 + * AssertionError when listing sstable files on inconsistent disk state (CASSANDRA-11156)
 + * Fix wrong rack counting and invalid conditions check for TokenAllocation
 +   (CASSANDRA-11139)
 + * Avoid creating empty hint files (CASSANDRA-11090)
 + * Fix leak detection strong reference loop using weak reference (CASSANDRA-11120)
 + * Configurie BatchlogManager to stop delayed tasks on shutdown (CASSANDRA-11062)
 + * Hadoop integration is incompatible with Cassandra Driver 3.0.0 (CASSANDRA-11001)
 + * Add dropped_columns to the list of schema table so it gets handled
 +   properly (CASSANDRA-11050)
 + * Fix NPE when using forceRepairRangeAsync without DC (CASSANDRA-11239)
 +Merged from 2.2:
   * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
   * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
   * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 7c46fcb,ca02747..f339660
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -434,10 -441,10 +448,10 @@@ public class CompactionManager implemen
              @Override
              public void execute(LifecycleTransaction txn) throws IOException
              {
 -                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
 +                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges, FBUtilities.nowInSeconds());
                  doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
+         }, jobs, OperationType.CLEANUP);
      }
  
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 3a62063,bca5996..75573ac
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2633,32 -2553,42 +2633,42 @@@ public class StorageService extends Not
          return Gossiper.instance.getCurrentGenerationNumber(FBUtilities.getBroadcastAddress());
      }
  
 -    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
++        return forceKeyspaceCleanup(0, keyspaceName, tables);
+     }
+ 
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
 -        if (keyspaceName.equals(SystemKeyspace.NAME))
 +        if (Schema.isSystemKeyspace(keyspaceName))
              throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
  
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup(jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
          return status.statusCode;
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, tables);
 -        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, tables);
+     }
+ 
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
@@@ -2677,12 -2606,17 +2687,17 @@@
          return status.statusCode;
      }
  
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
++        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 0, tableNames);
+     }
+ 
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, tableNames))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 049b4f9,761eed6..5b7331d
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -251,7 -251,9 +251,9 @@@ public interface StorageServiceMBean ex
      /**
       * Trigger a cleanup of keys on a single keyspace
       */
+     @Deprecated
 -    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException;
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@@ -260,8 -262,10 +262,10 @@@
       * Scrubbed CFs will be snapshotted first, if disableSnapshot is false
       */
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Verify (checksums of) the given keyspace.
@@@ -275,7 -279,9 +279,9 @@@
       * Rewrite all sstables to the latest version.
       * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
       */
+     @Deprecated
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException;
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Flush all memtables for the given column families, or all columnfamilies for the given keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index cdce497,2f27cea..a9d71d8
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -53,9 -53,9 +53,10 @@@ import javax.management.remote.JMXConne
  import javax.management.remote.JMXServiceURL;
  import javax.rmi.ssl.SslRMIClientSocketFactory;
  
 +import org.apache.cassandra.batchlog.BatchlogManager;
 +import org.apache.cassandra.batchlog.BatchlogManagerMBean;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 -import org.apache.cassandra.db.HintedHandOffManager;
  import org.apache.cassandra.db.HintedHandOffManagerMBean;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.compaction.CompactionManagerMBean;
@@@ -236,38 -231,46 +237,46 @@@ public class NodeProbe implements AutoC
          jmxc.close();
      }
  
-     public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.forceKeyspaceCleanup(keyspaceName, tables);
 -        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
++        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, tables);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, tables);
 -        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
++        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables);
      }
  
 -    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +        return ssProxy.verify(extendedVerify, keyspaceName, tableNames);
      }
  
-     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, tableNames);
 -        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
++        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames);
      }
  
-     public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     private void checkJobs(PrintStream out, int jobs)
      {
-         if (forceKeyspaceCleanup(keyspaceName, tableNames) != 0)
+         if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+             out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+     }
+ 
 -    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     {
+         checkJobs(out, jobs);
 -        if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
++        if (forceKeyspaceCleanup(jobs, keyspaceName, tableNames) != 0)
          {
              failed = true;
              out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, tables) != 0)
+         checkJobs(out, jobs);
 -        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
++        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables) != 0)
          {
              failed = true;
              out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
@@@ -284,9 -287,10 +293,10 @@@
      }
  
  
-     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
 -    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
-         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, tableNames) != 0)
+         checkJobs(out, jobs);
 -        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
++        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames) != 0)
          {
              failed = true;
              out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index cbfd4d6,6c6676d..a7507b1
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@@ -23,8 -23,8 +23,8 @@@ import io.airlift.command.Command
  import java.util.ArrayList;
  import java.util.List;
  
 +import org.apache.cassandra.config.Schema;
- import org.apache.cassandra.db.SystemKeyspace;
+ import io.airlift.command.Option;
 -import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.tools.NodeProbe;
  import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
  
@@@ -47,9 -52,8 +52,9 @@@ public class Cleanup extends NodeToolCm
  
              try
              {
-                 probe.forceKeyspaceCleanup(System.out, keyspace, tableNames);
 -                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
 -            } catch (Exception e)
++                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, tableNames);
 +            }
 +            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during cleanup", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index f92ae0d,dafe8d1..2345a85
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -58,11 -63,11 +63,13 @@@ public class Scrub extends NodeToolCm
          {
              try
              {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, tableNames);
 -                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
--            } catch (IllegalArgumentException e)
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, tableNames);
++            }
++            catch (IllegalArgumentException e)
              {
                  throw e;
--            } catch (Exception e)
++            }
++            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during scrubbing", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
index a775878,596f353..82866e0
--- a/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
@@@ -46,8 -51,8 +51,9 @@@ public class UpgradeSSTable extends Nod
          {
              try
              {
-                 probe.upgradeSSTables(System.out, keyspace, !includeAll, tableNames);
 -                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
--            } catch (Exception e)
++                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, tableNames);
++            }
++            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during enabling auto-compaction", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/CleanupTest.java
index 167f3b0,5777af4..b4ffe57
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@@ -92,11 -94,11 +92,11 @@@ public class CleanupTes
          // record max timestamps of the sstables pre-cleanup
          List<Long> expectedMaxTimestamps = getMaxTimestampList(cfs);
  
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(LOOPS, rows.size());
 +        iter = Util.getRangeSlice(cfs);
 +        assertEquals(LOOPS, Iterators.size(iter));
  
          // with one token in the ring, owned by the local node, cleanup should be a no-op
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
          // ensure max timestamp of the sstables are retained post-cleanup
          assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
@@@ -137,16 -143,18 +137,16 @@@
          tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
          tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
  
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
          // row data should be gone
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
  
          // not only should it be gone but there should be no data on disk, not even tombstones
 -        assert cfs.getSSTables().isEmpty();
 +        assert cfs.getLiveSSTables().isEmpty();
  
          // 2ary indexes should result in no results, too (although tombstones won't be gone until compacted)
 -        rows = cfs.search(range, clause, filter, Integer.MAX_VALUE);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).filterOn("birthdate", Operator.EQ, VALUE).build()).size());
      }
  
      @Test
@@@ -168,9 -180,10 +168,9 @@@
          tk2[0] = 1;
          tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
          tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
      }
  
      @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 6dbbb1b,4cca7ff..936ccd8
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -110,14 -127,18 +110,14 @@@ public class ScrubTes
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 1);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
      }
  
      @Test
@@@ -235,17 -260,21 +235,17 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 4);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(4, rows.size());
 +        assertOrderedAll(cfs, 4);
  
 -        SSTableReader sstable = cfs.getSSTables().iterator().next();
 +        SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
          overrideWithGarbage(sstable, 0, 2);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(4, rows.size());
 +        assertOrderedAll(cfs, 4);
      }
  
      @Test
@@@ -271,14 -318,18 +271,14 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 10);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
      }
  
      @Test
@@@ -289,17 -340,21 +289,17 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 10);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
  
 -        for (SSTableReader sstable : cfs.getSSTables())
 +        for (SSTableReader sstable : cfs.getLiveSSTables())
              new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)).delete();
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
      }
  
      @Test
@@@ -504,11 -599,13 +504,11 @@@
  
          QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
          ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
 -        Mutation mutation = new Mutation("Keyspace1", UTF8Type.instance.decompose("key"));
 -        CellNameType ct = cfs2.getComparator();
 -        mutation.add("test_scrub_validation", ct.makeCellName("b"), LongType.instance.decompose(1L), System.currentTimeMillis());
 -        mutation.apply();
 +
 +        new Mutation(UpdateBuilder.create(cfs2.metadata, "key").newRow().add("b", LongType.instance.decompose(1L)).build()).apply();
          cfs2.forceBlockingFlush();
  
-         CompactionManager.instance.performScrub(cfs2, false, false);
+         CompactionManager.instance.performScrub(cfs2, false, false, 2);
      }
  
      /**
@@@ -614,9 -763,9 +614,9 @@@
                  boolean failure = !scrubs[i];
                  if (failure)
                  { //make sure the next scrub fails
 -                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                    overrideWithGarbage(indexCfs.getLiveSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
                  }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true);
+                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
                  assertEquals(failure ?
                               CompactionManager.AllSSTableOpStatus.ABORTED :
                               CompactionManager.AllSSTableOpStatus.SUCCESSFUL,


[07/15] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

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


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

Branch: refs/heads/cassandra-2.2
Commit: 897ffe87e41ab128c9e8969d535cb2a706baf563
Parents: 6c1ef2b 8b8a3f5
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 10:54:45 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:54:45 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 16 ++++----
 .../db/compaction/CompactionManager.java        | 40 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 24 +++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 +++
 .../org/apache/cassandra/tools/NodeProbe.java   | 34 +++++++++++------
 .../cassandra/tools/nodetool/Cleanup.java       |  8 +++-
 .../apache/cassandra/tools/nodetool/Scrub.java  |  7 +++-
 .../tools/nodetool/UpgradeSSTable.java          |  7 +++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 20 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 12 files changed, 117 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 91dc588,7794d4f..098d062
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,39 -1,4 +1,41 @@@
 -2.1.14
 +2.2.6
 + * cqlsh: COPY FROM should check that explicit column names are valid (CASSANDRA-11333)
 + * Add -Dcassandra.start_gossip startup option (CASSANDRA-10809)
 + * Fix UTF8Validator.validate() for modified UTF-8 (CASSANDRA-10748)
 + * Clarify that now() function is calculated on the coordinator node in CQL documentation (CASSANDRA-10900)
 + * Fix bloom filter sizing with LCS (CASSANDRA-11344)
 + * (cqlsh) Fix error when result is 0 rows with EXPAND ON (CASSANDRA-11092)
 + * Fix intra-node serialization issue for multicolumn-restrictions (CASSANDRA-11196)
 + * Non-obsoleting compaction operations over compressed files can impose rate limit on normal reads (CASSANDRA-11301)
 + * Add missing newline at end of bin/cqlsh (CASSANDRA-11325)
 + * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 + * Fix filtering on non-primary key columns for thrift static column families
 +   (CASSANDRA-6377)
 + * Only log yaml config once, at startup (CASSANDRA-11217)
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 + * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
 + * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
 + * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
 + * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
 + * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
 + * Protect from keyspace dropped during repair (CASSANDRA-11065)
 + * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
 + * Better error message for cleanup (CASSANDRA-10991)
 + * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
 + * Use cloned TokenMetadata in size estimates to avoid race against membership check
 +   (CASSANDRA-10736)
 + * Always persist upsampled index summaries (CASSANDRA-10512)
 + * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
 + * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
 + * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
 + * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
 + * Fix paging on DISTINCT queries repeats result when first row in partition changes
 +   (CASSANDRA-10010)
 +Merged from 2.1:
++ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
++   many threads to use (CASSANDRA-11179)
   * Backport CASSANDRA-10679 (CASSANDRA-9598)
   * Don't do defragmentation if reading from repaired sstables (CASSANDRA-10342)
   * Fix streaming_socket_timeout_in_ms not enforced (CASSANDRA-11286)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index da4a84a,3d66d3a..09f58ac
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1499,74 -1478,22 +1499,74 @@@ public class ColumnFamilyStore implemen
          return maxFile;
      }
  
-     public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData);
++        return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 +    }
 +
 +    @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 +    {
          // skip snapshot creation during scrub, SEE JIRA 5891
          if(!disableSnapshot)
              snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
 -        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 +
 +        try
 +        {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 +        }
 +        catch(Throwable t)
 +        {
 +            if (!rebuildOnFailedScrub(t))
 +                throw t;
 +
 +            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        }
 +    }
 +
 +    /**
 +     * CASSANDRA-5174 : For an index cfs we may be able to discard everything and just rebuild
 +     * the index when a scrub fails.
 +     *
 +     * @return true if we are an index cfs and we successfully rebuilt the index
 +     */
 +    public boolean rebuildOnFailedScrub(Throwable failure)
 +    {
 +        if (!isIndex())
 +            return false;
 +
 +        SecondaryIndex index = null;
 +        if (metadata.cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR))
 +        {
 +            String[] parts = metadata.cfName.split("\\" + Directories.SECONDARY_INDEX_NAME_SEPARATOR, 2);
 +            ColumnFamilyStore parentCfs = keyspace.getColumnFamilyStore(parts[0]);
 +            index = parentCfs.indexManager.getIndexByName(metadata.cfName);
 +            assert index != null;
 +        }
 +
 +        if (index == null)
 +            return false;
 +
 +        truncateBlocking();
 +
 +        logger.warn("Rebuilding index for {} because of <{}>", name, failure.getMessage());
 +        index.getBaseCfs().rebuildSecondaryIndex(index.getIndexName());
 +        return true;
 +    }
 +
 +    public CompactionManager.AllSSTableOpStatus verify(boolean extendedVerify) throws ExecutionException, InterruptedException
 +    {
 +        return CompactionManager.instance.performVerify(ColumnFamilyStore.this, extendedVerify);
      }
  
-     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
      }
  
      public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index b015bcd,e382cab..ca02747
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -263,21 -271,37 +263,31 @@@ public class CompactionManager implemen
          }
      }
  
+     /**
+      * Run an operation over all sstables using jobs threads
+      *
+      * @param cfs the column family store to run the operation on
+      * @param operation the operation to run
+      * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+      * @return status of the operation
+      * @throws ExecutionException
+      * @throws InterruptedException
+      */
 -    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
 +    @SuppressWarnings("resource")
-     private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, OperationType operationType) throws ExecutionException, InterruptedException
++    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs, OperationType operationType) throws ExecutionException, InterruptedException
      {
 -        Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
 -        if (compactingSSTables == null)
 +        List<LifecycleTransaction> transactions = new ArrayList<>();
 +        try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType))
          {
 -            logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
 -            return AllSSTableOpStatus.ABORTED;
 -        }
 -        if (Iterables.isEmpty(compactingSSTables))
 -        {
 -            logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
 -            return AllSSTableOpStatus.SUCCESSFUL;
 -        }
 -        Set<SSTableReader> sstables = Sets.newHashSet(operation.filterSSTables(compactingSSTables));
 -        Set<SSTableReader> filteredAway = Sets.difference(Sets.newHashSet(compactingSSTables), sstables);
 -        cfs.getDataTracker().unmarkCompacting(filteredAway);
 -        final Set<SSTableReader> finished = Sets.newConcurrentHashSet();
 +            Iterable<SSTableReader> sstables = compacting != null ? Lists.newArrayList(operation.filterSSTables(compacting)) : Collections.<SSTableReader>emptyList();
 +            if (Iterables.isEmpty(sstables))
 +            {
 +                logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
 +                return AllSSTableOpStatus.SUCCESSFUL;
 +            }
 +
 +            List<Future<Object>> futures = new ArrayList<>();
  
 -        List<Future<Object>> futures = new ArrayList<>();
 -        try
 -        {
              for (final SSTableReader sstable : sstables)
              {
                  if (executor.isShutdown())
@@@ -286,23 -310,31 +296,27 @@@
                      return AllSSTableOpStatus.ABORTED;
                  }
  
 +                final LifecycleTransaction txn = compacting.split(singleton(sstable));
 +                transactions.add(txn);
-                 futures.add(executor.submit(new Callable<Object>()
+                 Callable<Object> callable = new Callable<Object>()
                  {
                      @Override
                      public Object call() throws Exception
                      {
 -                        try
 -                        {
 -                            operation.execute(sstable);
 -                        }
 -                        finally
 -                        {
 -                            cfs.getDataTracker().unmarkCompacting(Collections.singleton(sstable));
 -                            finished.add(sstable);
 -                        }
 +                        operation.execute(txn);
                          return this;
                      }
-                 }));
+                 };
+                 futures.add(executor.submit(callable));
+                 if (jobs > 0 && futures.size() == jobs)
+                 {
+                     FBUtilities.waitOnFutures(futures);
+                     futures.clear();
+                 }
              }
- 
-             assert compacting.originals().isEmpty();
- 
              FBUtilities.waitOnFutures(futures);
++            assert compacting.originals().isEmpty();
 +            return AllSSTableOpStatus.SUCCESSFUL;
          }
          finally
          {
@@@ -327,9 -358,9 +341,9 @@@
          }
      }
  
-     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData)
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
 +    throws InterruptedException, ExecutionException
      {
 -        assert !cfs.isIndex();
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
              @Override
@@@ -343,29 -374,10 +357,29 @@@
              {
                  scrubOne(cfs, input, skipCorrupted, checkData);
              }
-         }, OperationType.SCRUB);
 -        }, jobs);
++        }, jobs, OperationType.SCRUB);
 +    }
 +
 +    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
 +    {
 +        assert !cfs.isIndex();
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
 +            {
 +                return input.originals();
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction input) throws IOException
 +            {
 +                verifyOne(cfs, input.onlyOne(), extendedVerify);
 +            }
-         }, OperationType.VERIFY);
++        }, 0, OperationType.VERIFY);
      }
  
-     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
      {
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
@@@ -394,10 -402,10 +408,10 @@@
                  task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                  task.execute(metrics);
              }
-         }, OperationType.UPGRADE_SSTABLES);
 -        }, jobs);
++        }, jobs, OperationType.UPGRADE_SSTABLES);
      }
  
-     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
      {
          assert !cfStore.isIndex();
          Keyspace keyspace = cfStore.keyspace;
@@@ -425,12 -428,12 +439,12 @@@
              }
  
              @Override
 -            public void execute(SSTableReader input) throws IOException
 +            public void execute(LifecycleTransaction txn) throws IOException
              {
                  CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
 -                doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
 +                doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
 -        }, jobs);
++        }, jobs, OperationType.CLEANUP);
      }
  
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 5d29a5a,507aedb..bca5996
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2555,7 -2385,12 +2555,12 @@@ public class StorageService extends Not
  
      public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+     }
+ 
+     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
 -        if (keyspaceName.equals(Keyspace.SYSTEM_KS))
 +        if (keyspaceName.equals(SystemKeyspace.NAME))
              throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
  
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
@@@ -2575,30 -2410,27 +2580,39 @@@
  
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+     }
+ 
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
          return status.statusCode;
      }
- 
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        {
 +            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.verify(extendedVerify);
 +            if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
 +                status = oneStatus;
 +        }
 +        return status.statusCode;
 +    }
 +
      public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+     }
+ 
+     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
          for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 7c5cd0a,d3a1725..761eed6
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -261,17 -272,11 +263,19 @@@ public interface StorageServiceMBean ex
       */
      @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
 +     * Verify (checksums of) the given keyspace.
 +     * If columnFamilies array is empty, all CFs are verified.
 +     *
 +     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
 +     */
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +
 +    /**
       * Rewrite all sstables to the latest version.
       * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 93804a8,ab08e9f..2f27cea
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -53,6 -62,8 +53,7 @@@ import javax.management.remote.JMXConne
  import javax.management.remote.JMXServiceURL;
  import javax.rmi.ssl.SslRMIClientSocketFactory;
  
 -import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStoreMBean;
  import org.apache.cassandra.db.HintedHandOffManager;
  import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@@ -230,60 -238,53 +231,69 @@@ public class NodeProbe implements AutoC
          jmxc.close();
      }
  
-     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
      }
  
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +    }
 +
-     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
      }
  
-     public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     private void checkJobs(PrintStream out, int jobs)
      {
-         if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+         if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+             out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+     }
+ 
+     public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
++        checkJobs(out, jobs);
+         if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+         checkJobs(out, jobs);
+         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +        }
 +    }
 +
 +    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
 +        {
 +            failed = true;
 +            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
 +
-     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+         checkJobs(out, jobs);
+         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index aa415b3,0000000..6c6676d
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@@ -1,56 -1,0 +1,62 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
++import io.airlift.command.Option;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "cleanup", description = "Triggers the immediate cleanup of keys no longer belonging to a node. By default, clean all keyspaces")
 +public class Cleanup extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to cleanup simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            if (SystemKeyspace.NAME.equals(keyspace))
 +                continue;
 +
 +            try
 +            {
-                 probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
++                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during cleanup", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 54f981e,0000000..dafe8d1
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -1,71 -1,0 +1,76 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
 +public class Scrub extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "disable_snapshot",
 +            name = {"-ns", "--no-snapshot"},
 +            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +    private boolean disableSnapshot = false;
 +
 +    @Option(title = "skip_corrupted",
 +            name = {"-s", "--skip-corrupted"},
 +            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +    private boolean skipCorrupted = false;
 +
 +    @Option(title = "no_validate",
 +                   name = {"-n", "--no-validate"},
 +                   description = "Do not validate columns using column validator")
 +    private boolean noValidation = false;
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
 +            } catch (IllegalArgumentException e)
 +            {
 +                throw e;
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during scrubbing", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
index 86a2cd5,0000000..596f353
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
@@@ -1,56 -1,0 +1,61 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "upgradesstables", description = "Rewrite sstables (for the requested tables) that are not on the current version (thus upgrading them to said current version)")
 +public class UpgradeSSTable extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
 +    private boolean includeAll = false;
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to upgrade simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
++                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during enabling auto-compaction", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index c0cde40,4efd082..4cca7ff
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -588,14 -499,14 +588,14 @@@ public class ScrubTes
      @Test
      public void testScrubColumnValidation() throws InterruptedException, RequestExecutionException, ExecutionException
      {
 -        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", ConsistencyLevel.ONE);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
  
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_static_columns");
  
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')", KEYSPACE));
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
          ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@@ -614,15 -525,15 +614,15 @@@
      @Test
      public void testColumnNameEqualToDefaultKeyAlias() throws ExecutionException, InterruptedException
      {
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 -        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("UUIDKeys");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_UUID);
  
 -        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "UUIDKeys");
 +        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF_UUID);
          cf.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
 -        Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
 +        Mutation mutation = new Mutation(KEYSPACE, ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
          mutation.applyUnsafe();
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          assertEquals(1, cfs.getSSTables().size());
      }
@@@ -634,19 -545,19 +634,19 @@@
      @Test
      public void testValidationCompactStorage() throws Exception
      {
 -        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", ConsistencyLevel.ONE);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", KEYSPACE), ConsistencyLevel.ONE);
  
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_dynamic_columns");
  
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')");
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')", KEYSPACE));
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')", KEYSPACE));
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')", KEYSPACE));
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, true, true);
+         CompactionManager.instance.performScrub(cfs, true, true, 2);
  
          // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
 -        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".test_compact_dynamic_columns", KEYSPACE));
          assertEquals(3, rs.size());
  
          Iterator<UntypedResultSet.Row> iter = rs.iterator();
@@@ -654,129 -565,4 +654,129 @@@
          assertEquals("bar", iter.next().getString("c"));
          assertEquals("boo", iter.next().getString("c"));
      }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
 +        // otherwise it uses LocalByPartitionerType
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
 +    }
 +
 +    /** The SecondaryIndex class is used for custom indexes so to avoid
 +     * making a public final field into a private field with getters
 +     * and setters, we resort to this hack in order to test it properly
 +     * since it can have two values which influence the scrubbing behavior.
 +     * @param comparator - the key comparator we want to test
 +     */
 +    private void setKeyComparator(AbstractType<?> comparator)
 +    {
 +        try
 +        {
 +            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
 +            keyComparator.setAccessible(true);
 +            int modifiers = keyComparator.getModifiers();
 +            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
 +            modifiers = modifiers & ~Modifier.FINAL;
 +            modifierField.setAccessible(true);
 +            modifierField.setInt(keyComparator, modifiers);
 +
 +            keyComparator.set(null, comparator);
 +        }
 +        catch (Exception ex)
 +        {
 +            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
 +            ex.printStackTrace();
 +        }
 +    }
 +
 +    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
 +            throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.instance.disableAutoCompaction();
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 +        cfs.clearUnsafe();
 +
 +        int numRows = 1000;
 +        long[] colValues = new long [numRows * 2]; // each row has two columns
 +        for (int i = 0; i < colValues.length; i+=2)
 +        {
 +            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
 +            colValues[i+1] = 3L; //other column
 +        }
 +        fillIndexCF(cfs, composite, colValues);
 +
 +        // check index
 +        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
 +        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +
 +        // scrub index
 +        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
 +        assertTrue(indexCfss.size() == 1);
 +        for(ColumnFamilyStore indexCfs : indexCfss)
 +        {
 +            for (int i = 0; i < scrubs.length; i++)
 +            {
 +                boolean failure = !scrubs[i];
 +                if (failure)
 +                { //make sure the next scrub fails
 +                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
 +                assertEquals(failure ?
 +                             CompactionManager.AllSSTableOpStatus.ABORTED :
 +                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
 +                                result);
 +            }
 +        }
 +
 +
 +        // check index is still working
 +        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------


[02/15] cassandra git commit: Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Posted by ma...@apache.org.
Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Patch by marcuse; reviewed by Carl Yeksigian for CASSANDRA-11179


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

Branch: refs/heads/cassandra-2.2
Commit: 8b8a3f5b99fa7a8eefed14cd7e41b81773617046
Parents: a9b5422
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Mar 14 11:01:11 2016 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:50:38 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 12 +++----
 .../db/compaction/CompactionManager.java        | 35 +++++++++++++++-----
 .../cassandra/service/StorageService.java       | 24 +++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 ++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 33 +++++++++++-------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++--
 .../org/apache/cassandra/db/CleanupTest.java    |  6 ++--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 18 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 10 files changed, 111 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5f01114..7794d4f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,6 +17,8 @@
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)
  * Avoid major compaction mixing repaired and unrepaired sstables in DTCS (CASSANDRA-11113)
  * test_bulk_round_trip_blogposts is failing occasionally (CASSANDRA-10938)
+ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
+   many threads to use (CASSANDRA-11179)
 
 
 2.1.13

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 a78f33f..3d66d3a 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1478,22 +1478,22 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return maxFile;
     }
 
-    public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
     }
 
     public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 ec7cb45..e382cab 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -271,7 +271,17 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation) throws ExecutionException, InterruptedException
+    /**
+     * Run an operation over all sstables using jobs threads
+     *
+     * @param cfs the column family store to run the operation on
+     * @param operation the operation to run
+     * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+     * @return status of the operation
+     * @throws ExecutionException
+     * @throws InterruptedException
+     */
+    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
     {
         Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
         if (compactingSSTables == null)
@@ -299,7 +309,8 @@ public class CompactionManager implements CompactionManagerMBean
                     logger.info("Executor has shut down, not submitting task");
                     return AllSSTableOpStatus.ABORTED;
                 }
-                futures.add(executor.submit(new Callable<Object>()
+
+                Callable<Object> callable = new Callable<Object>()
                 {
                     @Override
                     public Object call() throws Exception
@@ -315,7 +326,13 @@ public class CompactionManager implements CompactionManagerMBean
                         }
                         return this;
                     }
-                }));
+                };
+                futures.add(executor.submit(callable));
+                if (jobs > 0 && futures.size() == jobs)
+                {
+                    FBUtilities.waitOnFutures(futures);
+                    futures.clear();
+                }
             }
             FBUtilities.waitOnFutures(futures);
         }
@@ -341,7 +358,7 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
@@ -357,10 +374,10 @@ public class CompactionManager implements CompactionManagerMBean
             {
                 scrubOne(cfs, input, skipCorrupted, checkData);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
     {
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -385,10 +402,10 @@ public class CompactionManager implements CompactionManagerMBean
                 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                 task.execute(metrics);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfStore.isIndex();
         Keyspace keyspace = cfStore.keyspace;
@@ -416,7 +433,7 @@ public class CompactionManager implements CompactionManagerMBean
                 CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
                 doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
             }
-        });
+        }, jobs);
     }
 
     public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 98e2251..507aedb 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2385,13 +2385,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+    }
+
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         if (keyspaceName.equals(Keyspace.SYSTEM_KS))
             throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
 
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup(jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
@@ -2400,27 +2405,36 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, columnFamilies);
+        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
     }
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
         return status.statusCode;
     }
-
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+    }
+
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8fa2433..d3a1725 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -260,7 +260,9 @@ public interface StorageServiceMBean extends NotificationEmitter
     /**
      * Trigger a cleanup of keys on a single keyspace
      */
+    @Deprecated
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@ -270,13 +272,17 @@ public interface StorageServiceMBean extends NotificationEmitter
      */
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
      */
+    @Deprecated
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Flush all memtables for the given column families, or all columnfamilies for the given keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 1ad1147..ab08e9f 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -63,6 +63,7 @@ import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.HintedHandOffManager;
 import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@ -237,42 +238,50 @@ public class NodeProbe implements AutoCloseable
         jmxc.close();
     }
 
-    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
     }
 
-    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
     }
 
-    public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    private void checkJobs(PrintStream out, int jobs)
     {
-        if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+        if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+            out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+    }
+
+    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
+        if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 819049e..1de2e20 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -1138,6 +1138,11 @@ public class NodeTool
         @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
         private List<String> args = new ArrayList<>();
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to cleanup simultaneusly, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1151,7 +1156,7 @@ public class NodeTool
 
                 try
                 {
-                    probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
+                    probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during cleanup", e);
@@ -1267,6 +1272,11 @@ public class NodeTool
                 description = "Do not validate columns using column validator")
         private boolean noValidation = false;
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1277,7 +1287,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -1345,6 +1355,11 @@ public class NodeTool
         @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
         private boolean includeAll = false;
 
+        @Option(title = "jobs",
+                name = {"-j","--jobs"},
+                description = "Number of sstables to upgrade simultaneously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1355,7 +1370,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
+                    probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during enabling auto-compaction", e);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 1d04dfa..7f54ed7 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -83,7 +83,7 @@ public class CleanupTest extends SchemaLoader
         assertEquals(LOOPS, rows.size());
 
         // with one token in the ring, owned by the local node, cleanup should be a no-op
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // ensure max timestamp of the sstables are retained post-cleanup
         assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
@@ -128,7 +128,7 @@ public class CleanupTest extends SchemaLoader
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
 
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // row data should be gone
         rows = Util.getRangeSlice(cfs);
@@ -165,7 +165,7 @@ public class CleanupTest extends SchemaLoader
         tk2[0] = 1;
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         rows = Util.getRangeSlice(cfs);
         assertEquals(0, rows.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 167671b..4efd082 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -99,7 +99,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(1, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -228,7 +228,7 @@ public class ScrubTest extends SchemaLoader
         SSTableReader sstable = cfs.getSSTables().iterator().next();
         overrideWithGarbage(sstable, 0, 2);
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -249,7 +249,7 @@ public class ScrubTest extends SchemaLoader
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
         assert cfs.getSSTables().isEmpty();
     }
 
@@ -268,7 +268,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(10, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -293,7 +293,7 @@ public class ScrubTest extends SchemaLoader
         for (SSTableReader sstable : cfs.getSSTables())
             new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)).delete();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -506,7 +506,7 @@ public class ScrubTest extends SchemaLoader
 
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
         ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@ -516,7 +516,7 @@ public class ScrubTest extends SchemaLoader
         mutation.apply();
         cfs2.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs2, false, false);
+        CompactionManager.instance.performScrub(cfs2, false, false, 2);
     }
 
     /**
@@ -533,7 +533,7 @@ public class ScrubTest extends SchemaLoader
         Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
         mutation.applyUnsafe();
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         assertEquals(1, cfs.getSSTables().size());
     }
@@ -554,7 +554,7 @@ public class ScrubTest extends SchemaLoader
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, true, true);
+        CompactionManager.instance.performScrub(cfs, true, true, 2);
 
         // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
         UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 7d33c11..749056c 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -328,7 +328,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
         cfs.disableAutoCompaction();
-        cfs.sstablesRewrite(false);
+        cfs.sstablesRewrite(false, 2);
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
     }


[08/15] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

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


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

Branch: refs/heads/cassandra-3.0
Commit: 897ffe87e41ab128c9e8969d535cb2a706baf563
Parents: 6c1ef2b 8b8a3f5
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 10:54:45 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:54:45 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 16 ++++----
 .../db/compaction/CompactionManager.java        | 40 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 24 +++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 +++
 .../org/apache/cassandra/tools/NodeProbe.java   | 34 +++++++++++------
 .../cassandra/tools/nodetool/Cleanup.java       |  8 +++-
 .../apache/cassandra/tools/nodetool/Scrub.java  |  7 +++-
 .../tools/nodetool/UpgradeSSTable.java          |  7 +++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 20 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 12 files changed, 117 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 91dc588,7794d4f..098d062
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,39 -1,4 +1,41 @@@
 -2.1.14
 +2.2.6
 + * cqlsh: COPY FROM should check that explicit column names are valid (CASSANDRA-11333)
 + * Add -Dcassandra.start_gossip startup option (CASSANDRA-10809)
 + * Fix UTF8Validator.validate() for modified UTF-8 (CASSANDRA-10748)
 + * Clarify that now() function is calculated on the coordinator node in CQL documentation (CASSANDRA-10900)
 + * Fix bloom filter sizing with LCS (CASSANDRA-11344)
 + * (cqlsh) Fix error when result is 0 rows with EXPAND ON (CASSANDRA-11092)
 + * Fix intra-node serialization issue for multicolumn-restrictions (CASSANDRA-11196)
 + * Non-obsoleting compaction operations over compressed files can impose rate limit on normal reads (CASSANDRA-11301)
 + * Add missing newline at end of bin/cqlsh (CASSANDRA-11325)
 + * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 + * Fix filtering on non-primary key columns for thrift static column families
 +   (CASSANDRA-6377)
 + * Only log yaml config once, at startup (CASSANDRA-11217)
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 + * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
 + * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
 + * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
 + * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
 + * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
 + * Protect from keyspace dropped during repair (CASSANDRA-11065)
 + * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
 + * Better error message for cleanup (CASSANDRA-10991)
 + * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
 + * Use cloned TokenMetadata in size estimates to avoid race against membership check
 +   (CASSANDRA-10736)
 + * Always persist upsampled index summaries (CASSANDRA-10512)
 + * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
 + * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
 + * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
 + * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
 + * Fix paging on DISTINCT queries repeats result when first row in partition changes
 +   (CASSANDRA-10010)
 +Merged from 2.1:
++ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
++   many threads to use (CASSANDRA-11179)
   * Backport CASSANDRA-10679 (CASSANDRA-9598)
   * Don't do defragmentation if reading from repaired sstables (CASSANDRA-10342)
   * Fix streaming_socket_timeout_in_ms not enforced (CASSANDRA-11286)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index da4a84a,3d66d3a..09f58ac
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1499,74 -1478,22 +1499,74 @@@ public class ColumnFamilyStore implemen
          return maxFile;
      }
  
-     public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData);
++        return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 +    }
 +
 +    @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 +    {
          // skip snapshot creation during scrub, SEE JIRA 5891
          if(!disableSnapshot)
              snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
 -        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 +
 +        try
 +        {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 +        }
 +        catch(Throwable t)
 +        {
 +            if (!rebuildOnFailedScrub(t))
 +                throw t;
 +
 +            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        }
 +    }
 +
 +    /**
 +     * CASSANDRA-5174 : For an index cfs we may be able to discard everything and just rebuild
 +     * the index when a scrub fails.
 +     *
 +     * @return true if we are an index cfs and we successfully rebuilt the index
 +     */
 +    public boolean rebuildOnFailedScrub(Throwable failure)
 +    {
 +        if (!isIndex())
 +            return false;
 +
 +        SecondaryIndex index = null;
 +        if (metadata.cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR))
 +        {
 +            String[] parts = metadata.cfName.split("\\" + Directories.SECONDARY_INDEX_NAME_SEPARATOR, 2);
 +            ColumnFamilyStore parentCfs = keyspace.getColumnFamilyStore(parts[0]);
 +            index = parentCfs.indexManager.getIndexByName(metadata.cfName);
 +            assert index != null;
 +        }
 +
 +        if (index == null)
 +            return false;
 +
 +        truncateBlocking();
 +
 +        logger.warn("Rebuilding index for {} because of <{}>", name, failure.getMessage());
 +        index.getBaseCfs().rebuildSecondaryIndex(index.getIndexName());
 +        return true;
 +    }
 +
 +    public CompactionManager.AllSSTableOpStatus verify(boolean extendedVerify) throws ExecutionException, InterruptedException
 +    {
 +        return CompactionManager.instance.performVerify(ColumnFamilyStore.this, extendedVerify);
      }
  
-     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
      }
  
      public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index b015bcd,e382cab..ca02747
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -263,21 -271,37 +263,31 @@@ public class CompactionManager implemen
          }
      }
  
+     /**
+      * Run an operation over all sstables using jobs threads
+      *
+      * @param cfs the column family store to run the operation on
+      * @param operation the operation to run
+      * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+      * @return status of the operation
+      * @throws ExecutionException
+      * @throws InterruptedException
+      */
 -    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
 +    @SuppressWarnings("resource")
-     private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, OperationType operationType) throws ExecutionException, InterruptedException
++    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs, OperationType operationType) throws ExecutionException, InterruptedException
      {
 -        Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
 -        if (compactingSSTables == null)
 +        List<LifecycleTransaction> transactions = new ArrayList<>();
 +        try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType))
          {
 -            logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
 -            return AllSSTableOpStatus.ABORTED;
 -        }
 -        if (Iterables.isEmpty(compactingSSTables))
 -        {
 -            logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
 -            return AllSSTableOpStatus.SUCCESSFUL;
 -        }
 -        Set<SSTableReader> sstables = Sets.newHashSet(operation.filterSSTables(compactingSSTables));
 -        Set<SSTableReader> filteredAway = Sets.difference(Sets.newHashSet(compactingSSTables), sstables);
 -        cfs.getDataTracker().unmarkCompacting(filteredAway);
 -        final Set<SSTableReader> finished = Sets.newConcurrentHashSet();
 +            Iterable<SSTableReader> sstables = compacting != null ? Lists.newArrayList(operation.filterSSTables(compacting)) : Collections.<SSTableReader>emptyList();
 +            if (Iterables.isEmpty(sstables))
 +            {
 +                logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
 +                return AllSSTableOpStatus.SUCCESSFUL;
 +            }
 +
 +            List<Future<Object>> futures = new ArrayList<>();
  
 -        List<Future<Object>> futures = new ArrayList<>();
 -        try
 -        {
              for (final SSTableReader sstable : sstables)
              {
                  if (executor.isShutdown())
@@@ -286,23 -310,31 +296,27 @@@
                      return AllSSTableOpStatus.ABORTED;
                  }
  
 +                final LifecycleTransaction txn = compacting.split(singleton(sstable));
 +                transactions.add(txn);
-                 futures.add(executor.submit(new Callable<Object>()
+                 Callable<Object> callable = new Callable<Object>()
                  {
                      @Override
                      public Object call() throws Exception
                      {
 -                        try
 -                        {
 -                            operation.execute(sstable);
 -                        }
 -                        finally
 -                        {
 -                            cfs.getDataTracker().unmarkCompacting(Collections.singleton(sstable));
 -                            finished.add(sstable);
 -                        }
 +                        operation.execute(txn);
                          return this;
                      }
-                 }));
+                 };
+                 futures.add(executor.submit(callable));
+                 if (jobs > 0 && futures.size() == jobs)
+                 {
+                     FBUtilities.waitOnFutures(futures);
+                     futures.clear();
+                 }
              }
- 
-             assert compacting.originals().isEmpty();
- 
              FBUtilities.waitOnFutures(futures);
++            assert compacting.originals().isEmpty();
 +            return AllSSTableOpStatus.SUCCESSFUL;
          }
          finally
          {
@@@ -327,9 -358,9 +341,9 @@@
          }
      }
  
-     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData)
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
 +    throws InterruptedException, ExecutionException
      {
 -        assert !cfs.isIndex();
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
              @Override
@@@ -343,29 -374,10 +357,29 @@@
              {
                  scrubOne(cfs, input, skipCorrupted, checkData);
              }
-         }, OperationType.SCRUB);
 -        }, jobs);
++        }, jobs, OperationType.SCRUB);
 +    }
 +
 +    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
 +    {
 +        assert !cfs.isIndex();
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
 +            {
 +                return input.originals();
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction input) throws IOException
 +            {
 +                verifyOne(cfs, input.onlyOne(), extendedVerify);
 +            }
-         }, OperationType.VERIFY);
++        }, 0, OperationType.VERIFY);
      }
  
-     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
      {
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
@@@ -394,10 -402,10 +408,10 @@@
                  task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                  task.execute(metrics);
              }
-         }, OperationType.UPGRADE_SSTABLES);
 -        }, jobs);
++        }, jobs, OperationType.UPGRADE_SSTABLES);
      }
  
-     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
      {
          assert !cfStore.isIndex();
          Keyspace keyspace = cfStore.keyspace;
@@@ -425,12 -428,12 +439,12 @@@
              }
  
              @Override
 -            public void execute(SSTableReader input) throws IOException
 +            public void execute(LifecycleTransaction txn) throws IOException
              {
                  CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
 -                doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
 +                doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
 -        }, jobs);
++        }, jobs, OperationType.CLEANUP);
      }
  
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 5d29a5a,507aedb..bca5996
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2555,7 -2385,12 +2555,12 @@@ public class StorageService extends Not
  
      public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+     }
+ 
+     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
 -        if (keyspaceName.equals(Keyspace.SYSTEM_KS))
 +        if (keyspaceName.equals(SystemKeyspace.NAME))
              throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
  
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
@@@ -2575,30 -2410,27 +2580,39 @@@
  
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+     }
+ 
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
          return status.statusCode;
      }
- 
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        {
 +            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.verify(extendedVerify);
 +            if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
 +                status = oneStatus;
 +        }
 +        return status.statusCode;
 +    }
 +
      public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+     }
+ 
+     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
          for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 7c5cd0a,d3a1725..761eed6
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -261,17 -272,11 +263,19 @@@ public interface StorageServiceMBean ex
       */
      @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
 +     * Verify (checksums of) the given keyspace.
 +     * If columnFamilies array is empty, all CFs are verified.
 +     *
 +     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
 +     */
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +
 +    /**
       * Rewrite all sstables to the latest version.
       * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 93804a8,ab08e9f..2f27cea
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -53,6 -62,8 +53,7 @@@ import javax.management.remote.JMXConne
  import javax.management.remote.JMXServiceURL;
  import javax.rmi.ssl.SslRMIClientSocketFactory;
  
 -import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStoreMBean;
  import org.apache.cassandra.db.HintedHandOffManager;
  import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@@ -230,60 -238,53 +231,69 @@@ public class NodeProbe implements AutoC
          jmxc.close();
      }
  
-     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
      }
  
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +    }
 +
-     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
      }
  
-     public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     private void checkJobs(PrintStream out, int jobs)
      {
-         if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+         if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+             out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+     }
+ 
+     public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
++        checkJobs(out, jobs);
+         if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+         checkJobs(out, jobs);
+         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +        }
 +    }
 +
 +    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
 +        {
 +            failed = true;
 +            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
 +
-     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+         checkJobs(out, jobs);
+         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index aa415b3,0000000..6c6676d
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@@ -1,56 -1,0 +1,62 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
++import io.airlift.command.Option;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "cleanup", description = "Triggers the immediate cleanup of keys no longer belonging to a node. By default, clean all keyspaces")
 +public class Cleanup extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to cleanup simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            if (SystemKeyspace.NAME.equals(keyspace))
 +                continue;
 +
 +            try
 +            {
-                 probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
++                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during cleanup", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 54f981e,0000000..dafe8d1
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -1,71 -1,0 +1,76 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
 +public class Scrub extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "disable_snapshot",
 +            name = {"-ns", "--no-snapshot"},
 +            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +    private boolean disableSnapshot = false;
 +
 +    @Option(title = "skip_corrupted",
 +            name = {"-s", "--skip-corrupted"},
 +            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +    private boolean skipCorrupted = false;
 +
 +    @Option(title = "no_validate",
 +                   name = {"-n", "--no-validate"},
 +                   description = "Do not validate columns using column validator")
 +    private boolean noValidation = false;
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
 +            } catch (IllegalArgumentException e)
 +            {
 +                throw e;
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during scrubbing", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
index 86a2cd5,0000000..596f353
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
@@@ -1,56 -1,0 +1,61 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "upgradesstables", description = "Rewrite sstables (for the requested tables) that are not on the current version (thus upgrading them to said current version)")
 +public class UpgradeSSTable extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
 +    private boolean includeAll = false;
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to upgrade simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
++                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during enabling auto-compaction", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index c0cde40,4efd082..4cca7ff
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -588,14 -499,14 +588,14 @@@ public class ScrubTes
      @Test
      public void testScrubColumnValidation() throws InterruptedException, RequestExecutionException, ExecutionException
      {
 -        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", ConsistencyLevel.ONE);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
  
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_static_columns");
  
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')", KEYSPACE));
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
          ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@@ -614,15 -525,15 +614,15 @@@
      @Test
      public void testColumnNameEqualToDefaultKeyAlias() throws ExecutionException, InterruptedException
      {
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 -        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("UUIDKeys");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_UUID);
  
 -        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "UUIDKeys");
 +        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF_UUID);
          cf.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
 -        Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
 +        Mutation mutation = new Mutation(KEYSPACE, ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
          mutation.applyUnsafe();
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          assertEquals(1, cfs.getSSTables().size());
      }
@@@ -634,19 -545,19 +634,19 @@@
      @Test
      public void testValidationCompactStorage() throws Exception
      {
 -        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", ConsistencyLevel.ONE);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", KEYSPACE), ConsistencyLevel.ONE);
  
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_dynamic_columns");
  
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')");
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')", KEYSPACE));
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')", KEYSPACE));
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')", KEYSPACE));
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, true, true);
+         CompactionManager.instance.performScrub(cfs, true, true, 2);
  
          // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
 -        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".test_compact_dynamic_columns", KEYSPACE));
          assertEquals(3, rs.size());
  
          Iterator<UntypedResultSet.Row> iter = rs.iterator();
@@@ -654,129 -565,4 +654,129 @@@
          assertEquals("bar", iter.next().getString("c"));
          assertEquals("boo", iter.next().getString("c"));
      }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
 +        // otherwise it uses LocalByPartitionerType
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
 +    }
 +
 +    /** The SecondaryIndex class is used for custom indexes so to avoid
 +     * making a public final field into a private field with getters
 +     * and setters, we resort to this hack in order to test it properly
 +     * since it can have two values which influence the scrubbing behavior.
 +     * @param comparator - the key comparator we want to test
 +     */
 +    private void setKeyComparator(AbstractType<?> comparator)
 +    {
 +        try
 +        {
 +            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
 +            keyComparator.setAccessible(true);
 +            int modifiers = keyComparator.getModifiers();
 +            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
 +            modifiers = modifiers & ~Modifier.FINAL;
 +            modifierField.setAccessible(true);
 +            modifierField.setInt(keyComparator, modifiers);
 +
 +            keyComparator.set(null, comparator);
 +        }
 +        catch (Exception ex)
 +        {
 +            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
 +            ex.printStackTrace();
 +        }
 +    }
 +
 +    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
 +            throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.instance.disableAutoCompaction();
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 +        cfs.clearUnsafe();
 +
 +        int numRows = 1000;
 +        long[] colValues = new long [numRows * 2]; // each row has two columns
 +        for (int i = 0; i < colValues.length; i+=2)
 +        {
 +            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
 +            colValues[i+1] = 3L; //other column
 +        }
 +        fillIndexCF(cfs, composite, colValues);
 +
 +        // check index
 +        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
 +        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +
 +        // scrub index
 +        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
 +        assertTrue(indexCfss.size() == 1);
 +        for(ColumnFamilyStore indexCfs : indexCfss)
 +        {
 +            for (int i = 0; i < scrubs.length; i++)
 +            {
 +                boolean failure = !scrubs[i];
 +                if (failure)
 +                { //make sure the next scrub fails
 +                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
 +                assertEquals(failure ?
 +                             CompactionManager.AllSSTableOpStatus.ABORTED :
 +                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
 +                                result);
 +            }
 +        }
 +
 +
 +        // check index is still working
 +        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------


[14/15] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.5

Posted by ma...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.5


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

Branch: refs/heads/trunk
Commit: 0a2508544069f035dc43a81b8700c0b7bf2d28e3
Parents: c7ef7c9 0c91977
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 11:05:46 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 11:05:46 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 20 ++++-----
 .../db/compaction/CompactionManager.java        | 44 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 30 ++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  8 ++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 38 ++++++++++-------
 .../cassandra/tools/nodetool/Cleanup.java       |  9 +++-
 .../tools/nodetool/RelocateSSTables.java        |  8 +++-
 .../apache/cassandra/tools/nodetool/Scrub.java  | 13 ++++--
 .../tools/nodetool/UpgradeSSTable.java          | 10 ++++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 16 +++----
 12 files changed, 141 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 2907df9,fdc873e..3e69959
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -25,43 -23,14 +25,45 @@@ Merged from 2.2
   * Only log yaml config once, at startup (CASSANDRA-11217)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
  Merged from 2.1:
+  * Add a -j parameter to scrub/cleanup/upgradesstables to state how
+    many threads to use (CASSANDRA-11179)
 - * Backport CASSANDRA-10679 (CASSANDRA-9598)
 - * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
   * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  
 -3.0.4
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 +
 +3.4
 + * (cqlsh) add cqlshrc option to always connect using ssl (CASSANDRA-10458)
 + * Cleanup a few resource warnings (CASSANDRA-11085)
 + * Allow custom tracing implementations (CASSANDRA-10392)
 + * Extract LoaderOptions to be able to be used from outside (CASSANDRA-10637)
 + * fix OnDiskIndexTest to properly treat empty ranges (CASSANDRA-11205)
 + * fix TrackerTest to handle new notifications (CASSANDRA-11178)
 + * add SASI validation for partitioner and complex columns (CASSANDRA-11169)
 + * Add caching of encrypted credentials in PasswordAuthenticator (CASSANDRA-7715)
 + * fix SASI memtable switching on flush (CASSANDRA-11159)
 + * Remove duplicate offline compaction tracking (CASSANDRA-11148)
 + * fix EQ semantics of analyzed SASI indexes (CASSANDRA-11130)
 + * Support long name output for nodetool commands (CASSANDRA-7950)
 + * Encrypted hints (CASSANDRA-11040)
 + * SASI index options validation (CASSANDRA-11136)
 + * Optimize disk seek using min/max column name meta data when the LIMIT clause is used
 +   (CASSANDRA-8180)
 + * Add LIKE support to CQL3 (CASSANDRA-11067)
 + * Generic Java UDF types (CASSANDRA-10819)
 + * cqlsh: Include sub-second precision in timestamps by default (CASSANDRA-10428)
 + * Set javac encoding to utf-8 (CASSANDRA-11077)
 + * Integrate SASI index into Cassandra (CASSANDRA-10661)
 + * Add --skip-flush option to nodetool snapshot
 + * Skip values for non-queried columns (CASSANDRA-10657)
 + * Add support for secondary indexes on static columns (CASSANDRA-8103)
 + * CommitLogUpgradeTestMaker creates broken commit logs (CASSANDRA-11051)
 + * Add metric for number of dropped mutations (CASSANDRA-10866)
 + * Simplify row cache invalidation code (CASSANDRA-10396)
 + * Support user-defined compaction through nodetool (CASSANDRA-10660)
 + * Stripe view locks by key and table ID to reduce contention (CASSANDRA-10981)
 + * Add nodetool gettimeout and settimeout commands (CASSANDRA-10953)
 + * Add 3.0 metadata to sstablemetadata output (CASSANDRA-10838)
 +Merged from 3.0:
   * MV should only query complex columns included in the view (CASSANDRA-11069)
   * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
   * Add sstabledump tool (CASSANDRA-7464)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index fbfb7ee,a1db5b5..f193c4d
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1436,16 -1363,11 +1436,16 @@@ public class ColumnFamilyStore implemen
          return CompactionManager.instance.performVerify(ColumnFamilyStore.this, extendedVerify);
      }
  
-     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus relocateSSTables() throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus relocateSSTables(int jobs) throws ExecutionException, InterruptedException
 +    {
-         return CompactionManager.instance.relocateSSTables(this);
++        return CompactionManager.instance.relocateSSTables(this, jobs);
 +    }
 +
      public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
      {
          assert !sstables.isEmpty();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 627ae32,f339660..f9b0997
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -438,80 -451,9 +452,80 @@@ public class CompactionManager implemen
                  CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges, FBUtilities.nowInSeconds());
                  doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
+         }, jobs, OperationType.CLEANUP);
      }
  
-     public AllSSTableOpStatus relocateSSTables(final ColumnFamilyStore cfs) throws ExecutionException, InterruptedException
++    public AllSSTableOpStatus relocateSSTables(final ColumnFamilyStore cfs, int jobs) throws ExecutionException, InterruptedException
 +    {
 +        if (!cfs.getPartitioner().splitter().isPresent())
 +        {
 +            logger.info("Partitioner does not support splitting");
 +            return AllSSTableOpStatus.ABORTED;
 +        }
 +        final Collection<Range<Token>> r = StorageService.instance.getLocalRanges(cfs.keyspace.getName());
 +
 +        if (r.isEmpty())
 +        {
 +            logger.info("Relocate cannot run before a node has joined the ring");
 +            return AllSSTableOpStatus.ABORTED;
 +        }
 +
 +        final List<Range<Token>> localRanges = Range.sort(r);
 +        final Directories.DataDirectory[] locations = cfs.getDirectories().getWriteableLocations();
 +        final List<PartitionPosition> diskBoundaries = StorageService.getDiskBoundaries(localRanges, cfs.getPartitioner(), locations);
 +
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction transaction)
 +            {
 +                Set<SSTableReader> originals = Sets.newHashSet(transaction.originals());
 +                Set<SSTableReader> needsRelocation = originals.stream().filter(s -> !inCorrectLocation(s)).collect(Collectors.toSet());
 +                transaction.cancel(Sets.difference(originals, needsRelocation));
 +
 +                Map<Integer, List<SSTableReader>> groupedByDisk = needsRelocation.stream().collect(Collectors.groupingBy((s) ->
 +                        CompactionStrategyManager.getCompactionStrategyIndex(cfs, cfs.getDirectories(), s)));
 +
 +                int maxSize = 0;
 +                for (List<SSTableReader> diskSSTables : groupedByDisk.values())
 +                    maxSize = Math.max(maxSize, diskSSTables.size());
 +
 +                List<SSTableReader> mixedSSTables = new ArrayList<>();
 +
 +                for (int i = 0; i < maxSize; i++)
 +                    for (List<SSTableReader> diskSSTables : groupedByDisk.values())
 +                        if (i < diskSSTables.size())
 +                            mixedSSTables.add(diskSSTables.get(i));
 +
 +                return mixedSSTables;
 +            }
 +
 +            private boolean inCorrectLocation(SSTableReader sstable)
 +            {
 +                if (!cfs.getPartitioner().splitter().isPresent())
 +                    return true;
 +                int directoryIndex = CompactionStrategyManager.getCompactionStrategyIndex(cfs, cfs.getDirectories(), sstable);
 +                Directories.DataDirectory[] locations = cfs.getDirectories().getWriteableLocations();
 +
 +                Directories.DataDirectory location = locations[directoryIndex];
 +                PartitionPosition diskLast = diskBoundaries.get(directoryIndex);
 +                // the location we get from directoryIndex is based on the first key in the sstable
 +                // now we need to make sure the last key is less than the boundary as well:
 +                return sstable.descriptor.directory.getAbsolutePath().startsWith(location.location.getAbsolutePath()) && sstable.last.compareTo(diskLast) <= 0;
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction txn) throws IOException
 +            {
 +                logger.debug("Relocating {}", txn.originals());
 +                AbstractCompactionTask task = cfs.getCompactionStrategyManager().getCompactionTask(txn, NO_GC, Long.MAX_VALUE);
 +                task.setUserDefined(true);
 +                task.setCompactionType(OperationType.RELOCATE);
 +                task.execute(metrics);
 +            }
-         }, OperationType.RELOCATE);
++        }, jobs, OperationType.RELOCATE);
 +    }
 +
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,
                                            final Collection<Range<Token>> ranges,
                                            final Refs<SSTableReader> sstables,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 22d67e5,75573ac..52dcb97
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2789,59 -2712,6 +2804,64 @@@ public class StorageService extends Not
          }
      }
  
 +    public int relocateSSTables(String keyspaceName, String ... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
++        return relocateSSTables(0, keyspaceName, columnFamilies);
++    }
++
++    public int relocateSSTables(int jobs, String keyspaceName, String ... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    {
 +        CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        for (ColumnFamilyStore cfs : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfs.relocateSSTables();
++            CompactionManager.AllSSTableOpStatus oneStatus = cfs.relocateSSTables(jobs);
 +            if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
 +                status = oneStatus;
 +        }
 +        return status.statusCode;
 +    }
 +
 +    /**
 +     * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
 +     *
 +     * @param tag
 +     *            the tag given to the snapshot; may not be null or empty
 +     * @param options
 +     *            Map of options (skipFlush is the only supported option for now)
 +     * @param entities
 +     *            list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,...
 +     */
 +    @Override
 +    public void takeSnapshot(String tag, Map<String, String> options, String... entities) throws IOException
 +    {
 +        boolean skipFlush = Boolean.parseBoolean(options.getOrDefault("skipFlush", "false"));
 +
 +        if (entities != null && entities.length > 0 && entities[0].contains("."))
 +        {
 +            takeMultipleTableSnapshot(tag, skipFlush, entities);
 +        }
 +        else
 +        {
 +            takeSnapshot(tag, skipFlush, entities);
 +        }
 +    }
 +
 +    /**
 +     * Takes the snapshot of a specific table. A snapshot name must be
 +     * specified.
 +     *
 +     * @param keyspaceName
 +     *            the keyspace which holds the specified table
 +     * @param tableName
 +     *            the table to snapshot
 +     * @param tag
 +     *            the tag given to the snapshot; may not be null or empty
 +     */
 +    public void takeTableSnapshot(String keyspaceName, String tableName, String tag)
 +            throws IOException {
 +        takeMultipleTableSnapshot(tag, false, keyspaceName + "." + tableName);
 +    }
 +
      /**
       * Takes the snapshot for the given keyspaces. A snapshot name must be specified.
       *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index dc12253,5b7331d..2238f33
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -251,7 -248,6 +251,9 @@@ public interface StorageServiceMBean ex
       */
      public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
  
++    @Deprecated
 +    public int relocateSSTables(String keyspace, String ... cfnames) throws IOException, ExecutionException, InterruptedException;
++    public int relocateSSTables(int jobs, String keyspace, String ... cfnames) throws IOException, ExecutionException, InterruptedException;
      /**
       * Trigger a cleanup of keys on a single keyspace
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 405f70c,a9d71d8..7d15926
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -304,11 -309,6 +314,11 @@@ public class NodeProbe implements AutoC
          ssProxy.forceKeyspaceCompaction(splitOutput, keyspaceName, tableNames);
      }
  
-     public void relocateSSTables(String keyspace, String[] cfnames) throws IOException, ExecutionException, InterruptedException
++    public void relocateSSTables(int jobs, String keyspace, String[] cfnames) throws IOException, ExecutionException, InterruptedException
 +    {
-         ssProxy.relocateSSTables(keyspace, cfnames);
++        ssProxy.relocateSSTables(jobs, keyspace, cfnames);
 +    }
 +
      public void forceKeyspaceFlush(String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
          ssProxy.forceKeyspaceFlush(keyspaceName, tableNames);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/tools/nodetool/RelocateSSTables.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/RelocateSSTables.java
index 8522bc4,0000000..7c3066c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/RelocateSSTables.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/RelocateSSTables.java
@@@ -1,49 -1,0 +1,55 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
++import io.airlift.command.Option;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool;
 +
 +@Command(name = "relocatesstables", description = "Relocates sstables to the correct disk")
 +public class RelocateSSTables extends NodeTool.NodeToolCmd
 +{
 +    @Arguments(usage = "<keyspace> <table>", description = "The keyspace and table name")
 +    private List<String> args = new ArrayList<>();
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to relocate simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalTables(args);
 +        try
 +        {
 +            for (String keyspace : keyspaces)
-                 probe.relocateSSTables(keyspace, cfnames);
++                probe.relocateSSTables(jobs, keyspace, cfnames);
 +        }
 +        catch (Exception e)
 +        {
 +            throw new RuntimeException("Got error while relocating", e);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------


[04/15] cassandra git commit: Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Posted by ma...@apache.org.
Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Patch by marcuse; reviewed by Carl Yeksigian for CASSANDRA-11179


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

Branch: refs/heads/cassandra-3.0
Commit: 8b8a3f5b99fa7a8eefed14cd7e41b81773617046
Parents: a9b5422
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Mar 14 11:01:11 2016 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:50:38 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 12 +++----
 .../db/compaction/CompactionManager.java        | 35 +++++++++++++++-----
 .../cassandra/service/StorageService.java       | 24 +++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 ++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 33 +++++++++++-------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++--
 .../org/apache/cassandra/db/CleanupTest.java    |  6 ++--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 18 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 10 files changed, 111 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5f01114..7794d4f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,6 +17,8 @@
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)
  * Avoid major compaction mixing repaired and unrepaired sstables in DTCS (CASSANDRA-11113)
  * test_bulk_round_trip_blogposts is failing occasionally (CASSANDRA-10938)
+ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
+   many threads to use (CASSANDRA-11179)
 
 
 2.1.13

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 a78f33f..3d66d3a 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1478,22 +1478,22 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return maxFile;
     }
 
-    public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
     }
 
     public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 ec7cb45..e382cab 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -271,7 +271,17 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation) throws ExecutionException, InterruptedException
+    /**
+     * Run an operation over all sstables using jobs threads
+     *
+     * @param cfs the column family store to run the operation on
+     * @param operation the operation to run
+     * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+     * @return status of the operation
+     * @throws ExecutionException
+     * @throws InterruptedException
+     */
+    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
     {
         Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
         if (compactingSSTables == null)
@@ -299,7 +309,8 @@ public class CompactionManager implements CompactionManagerMBean
                     logger.info("Executor has shut down, not submitting task");
                     return AllSSTableOpStatus.ABORTED;
                 }
-                futures.add(executor.submit(new Callable<Object>()
+
+                Callable<Object> callable = new Callable<Object>()
                 {
                     @Override
                     public Object call() throws Exception
@@ -315,7 +326,13 @@ public class CompactionManager implements CompactionManagerMBean
                         }
                         return this;
                     }
-                }));
+                };
+                futures.add(executor.submit(callable));
+                if (jobs > 0 && futures.size() == jobs)
+                {
+                    FBUtilities.waitOnFutures(futures);
+                    futures.clear();
+                }
             }
             FBUtilities.waitOnFutures(futures);
         }
@@ -341,7 +358,7 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
@@ -357,10 +374,10 @@ public class CompactionManager implements CompactionManagerMBean
             {
                 scrubOne(cfs, input, skipCorrupted, checkData);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
     {
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -385,10 +402,10 @@ public class CompactionManager implements CompactionManagerMBean
                 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                 task.execute(metrics);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfStore.isIndex();
         Keyspace keyspace = cfStore.keyspace;
@@ -416,7 +433,7 @@ public class CompactionManager implements CompactionManagerMBean
                 CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
                 doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
             }
-        });
+        }, jobs);
     }
 
     public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 98e2251..507aedb 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2385,13 +2385,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+    }
+
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         if (keyspaceName.equals(Keyspace.SYSTEM_KS))
             throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
 
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup(jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
@@ -2400,27 +2405,36 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, columnFamilies);
+        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
     }
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
         return status.statusCode;
     }
-
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+    }
+
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8fa2433..d3a1725 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -260,7 +260,9 @@ public interface StorageServiceMBean extends NotificationEmitter
     /**
      * Trigger a cleanup of keys on a single keyspace
      */
+    @Deprecated
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@ -270,13 +272,17 @@ public interface StorageServiceMBean extends NotificationEmitter
      */
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
      */
+    @Deprecated
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Flush all memtables for the given column families, or all columnfamilies for the given keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 1ad1147..ab08e9f 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -63,6 +63,7 @@ import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.HintedHandOffManager;
 import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@ -237,42 +238,50 @@ public class NodeProbe implements AutoCloseable
         jmxc.close();
     }
 
-    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
     }
 
-    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
     }
 
-    public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    private void checkJobs(PrintStream out, int jobs)
     {
-        if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+        if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+            out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+    }
+
+    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
+        if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 819049e..1de2e20 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -1138,6 +1138,11 @@ public class NodeTool
         @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
         private List<String> args = new ArrayList<>();
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to cleanup simultaneusly, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1151,7 +1156,7 @@ public class NodeTool
 
                 try
                 {
-                    probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
+                    probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during cleanup", e);
@@ -1267,6 +1272,11 @@ public class NodeTool
                 description = "Do not validate columns using column validator")
         private boolean noValidation = false;
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1277,7 +1287,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -1345,6 +1355,11 @@ public class NodeTool
         @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
         private boolean includeAll = false;
 
+        @Option(title = "jobs",
+                name = {"-j","--jobs"},
+                description = "Number of sstables to upgrade simultaneously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1355,7 +1370,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
+                    probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during enabling auto-compaction", e);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 1d04dfa..7f54ed7 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -83,7 +83,7 @@ public class CleanupTest extends SchemaLoader
         assertEquals(LOOPS, rows.size());
 
         // with one token in the ring, owned by the local node, cleanup should be a no-op
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // ensure max timestamp of the sstables are retained post-cleanup
         assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
@@ -128,7 +128,7 @@ public class CleanupTest extends SchemaLoader
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
 
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // row data should be gone
         rows = Util.getRangeSlice(cfs);
@@ -165,7 +165,7 @@ public class CleanupTest extends SchemaLoader
         tk2[0] = 1;
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         rows = Util.getRangeSlice(cfs);
         assertEquals(0, rows.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 167671b..4efd082 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -99,7 +99,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(1, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -228,7 +228,7 @@ public class ScrubTest extends SchemaLoader
         SSTableReader sstable = cfs.getSSTables().iterator().next();
         overrideWithGarbage(sstable, 0, 2);
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -249,7 +249,7 @@ public class ScrubTest extends SchemaLoader
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
         assert cfs.getSSTables().isEmpty();
     }
 
@@ -268,7 +268,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(10, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -293,7 +293,7 @@ public class ScrubTest extends SchemaLoader
         for (SSTableReader sstable : cfs.getSSTables())
             new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)).delete();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -506,7 +506,7 @@ public class ScrubTest extends SchemaLoader
 
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
         ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@ -516,7 +516,7 @@ public class ScrubTest extends SchemaLoader
         mutation.apply();
         cfs2.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs2, false, false);
+        CompactionManager.instance.performScrub(cfs2, false, false, 2);
     }
 
     /**
@@ -533,7 +533,7 @@ public class ScrubTest extends SchemaLoader
         Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
         mutation.applyUnsafe();
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         assertEquals(1, cfs.getSSTables().size());
     }
@@ -554,7 +554,7 @@ public class ScrubTest extends SchemaLoader
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, true, true);
+        CompactionManager.instance.performScrub(cfs, true, true, 2);
 
         // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
         UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 7d33c11..749056c 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -328,7 +328,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
         cfs.disableAutoCompaction();
-        cfs.sstablesRewrite(false);
+        cfs.sstablesRewrite(false, 2);
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
     }


[09/15] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

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


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

Branch: refs/heads/cassandra-3.5
Commit: 897ffe87e41ab128c9e8969d535cb2a706baf563
Parents: 6c1ef2b 8b8a3f5
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 10:54:45 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:54:45 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 16 ++++----
 .../db/compaction/CompactionManager.java        | 40 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 24 +++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 +++
 .../org/apache/cassandra/tools/NodeProbe.java   | 34 +++++++++++------
 .../cassandra/tools/nodetool/Cleanup.java       |  8 +++-
 .../apache/cassandra/tools/nodetool/Scrub.java  |  7 +++-
 .../tools/nodetool/UpgradeSSTable.java          |  7 +++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 20 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 12 files changed, 117 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 91dc588,7794d4f..098d062
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,39 -1,4 +1,41 @@@
 -2.1.14
 +2.2.6
 + * cqlsh: COPY FROM should check that explicit column names are valid (CASSANDRA-11333)
 + * Add -Dcassandra.start_gossip startup option (CASSANDRA-10809)
 + * Fix UTF8Validator.validate() for modified UTF-8 (CASSANDRA-10748)
 + * Clarify that now() function is calculated on the coordinator node in CQL documentation (CASSANDRA-10900)
 + * Fix bloom filter sizing with LCS (CASSANDRA-11344)
 + * (cqlsh) Fix error when result is 0 rows with EXPAND ON (CASSANDRA-11092)
 + * Fix intra-node serialization issue for multicolumn-restrictions (CASSANDRA-11196)
 + * Non-obsoleting compaction operations over compressed files can impose rate limit on normal reads (CASSANDRA-11301)
 + * Add missing newline at end of bin/cqlsh (CASSANDRA-11325)
 + * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 + * Fix filtering on non-primary key columns for thrift static column families
 +   (CASSANDRA-6377)
 + * Only log yaml config once, at startup (CASSANDRA-11217)
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 + * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
 + * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
 + * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
 + * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
 + * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
 + * Protect from keyspace dropped during repair (CASSANDRA-11065)
 + * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
 + * Better error message for cleanup (CASSANDRA-10991)
 + * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
 + * Use cloned TokenMetadata in size estimates to avoid race against membership check
 +   (CASSANDRA-10736)
 + * Always persist upsampled index summaries (CASSANDRA-10512)
 + * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
 + * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
 + * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
 + * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
 + * Fix paging on DISTINCT queries repeats result when first row in partition changes
 +   (CASSANDRA-10010)
 +Merged from 2.1:
++ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
++   many threads to use (CASSANDRA-11179)
   * Backport CASSANDRA-10679 (CASSANDRA-9598)
   * Don't do defragmentation if reading from repaired sstables (CASSANDRA-10342)
   * Fix streaming_socket_timeout_in_ms not enforced (CASSANDRA-11286)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index da4a84a,3d66d3a..09f58ac
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1499,74 -1478,22 +1499,74 @@@ public class ColumnFamilyStore implemen
          return maxFile;
      }
  
-     public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData);
++        return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 +    }
 +
 +    @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 +    {
          // skip snapshot creation during scrub, SEE JIRA 5891
          if(!disableSnapshot)
              snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
 -        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 +
 +        try
 +        {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 +        }
 +        catch(Throwable t)
 +        {
 +            if (!rebuildOnFailedScrub(t))
 +                throw t;
 +
 +            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        }
 +    }
 +
 +    /**
 +     * CASSANDRA-5174 : For an index cfs we may be able to discard everything and just rebuild
 +     * the index when a scrub fails.
 +     *
 +     * @return true if we are an index cfs and we successfully rebuilt the index
 +     */
 +    public boolean rebuildOnFailedScrub(Throwable failure)
 +    {
 +        if (!isIndex())
 +            return false;
 +
 +        SecondaryIndex index = null;
 +        if (metadata.cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR))
 +        {
 +            String[] parts = metadata.cfName.split("\\" + Directories.SECONDARY_INDEX_NAME_SEPARATOR, 2);
 +            ColumnFamilyStore parentCfs = keyspace.getColumnFamilyStore(parts[0]);
 +            index = parentCfs.indexManager.getIndexByName(metadata.cfName);
 +            assert index != null;
 +        }
 +
 +        if (index == null)
 +            return false;
 +
 +        truncateBlocking();
 +
 +        logger.warn("Rebuilding index for {} because of <{}>", name, failure.getMessage());
 +        index.getBaseCfs().rebuildSecondaryIndex(index.getIndexName());
 +        return true;
 +    }
 +
 +    public CompactionManager.AllSSTableOpStatus verify(boolean extendedVerify) throws ExecutionException, InterruptedException
 +    {
 +        return CompactionManager.instance.performVerify(ColumnFamilyStore.this, extendedVerify);
      }
  
-     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
      }
  
      public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index b015bcd,e382cab..ca02747
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -263,21 -271,37 +263,31 @@@ public class CompactionManager implemen
          }
      }
  
+     /**
+      * Run an operation over all sstables using jobs threads
+      *
+      * @param cfs the column family store to run the operation on
+      * @param operation the operation to run
+      * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+      * @return status of the operation
+      * @throws ExecutionException
+      * @throws InterruptedException
+      */
 -    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
 +    @SuppressWarnings("resource")
-     private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, OperationType operationType) throws ExecutionException, InterruptedException
++    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs, OperationType operationType) throws ExecutionException, InterruptedException
      {
 -        Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
 -        if (compactingSSTables == null)
 +        List<LifecycleTransaction> transactions = new ArrayList<>();
 +        try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType))
          {
 -            logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
 -            return AllSSTableOpStatus.ABORTED;
 -        }
 -        if (Iterables.isEmpty(compactingSSTables))
 -        {
 -            logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
 -            return AllSSTableOpStatus.SUCCESSFUL;
 -        }
 -        Set<SSTableReader> sstables = Sets.newHashSet(operation.filterSSTables(compactingSSTables));
 -        Set<SSTableReader> filteredAway = Sets.difference(Sets.newHashSet(compactingSSTables), sstables);
 -        cfs.getDataTracker().unmarkCompacting(filteredAway);
 -        final Set<SSTableReader> finished = Sets.newConcurrentHashSet();
 +            Iterable<SSTableReader> sstables = compacting != null ? Lists.newArrayList(operation.filterSSTables(compacting)) : Collections.<SSTableReader>emptyList();
 +            if (Iterables.isEmpty(sstables))
 +            {
 +                logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
 +                return AllSSTableOpStatus.SUCCESSFUL;
 +            }
 +
 +            List<Future<Object>> futures = new ArrayList<>();
  
 -        List<Future<Object>> futures = new ArrayList<>();
 -        try
 -        {
              for (final SSTableReader sstable : sstables)
              {
                  if (executor.isShutdown())
@@@ -286,23 -310,31 +296,27 @@@
                      return AllSSTableOpStatus.ABORTED;
                  }
  
 +                final LifecycleTransaction txn = compacting.split(singleton(sstable));
 +                transactions.add(txn);
-                 futures.add(executor.submit(new Callable<Object>()
+                 Callable<Object> callable = new Callable<Object>()
                  {
                      @Override
                      public Object call() throws Exception
                      {
 -                        try
 -                        {
 -                            operation.execute(sstable);
 -                        }
 -                        finally
 -                        {
 -                            cfs.getDataTracker().unmarkCompacting(Collections.singleton(sstable));
 -                            finished.add(sstable);
 -                        }
 +                        operation.execute(txn);
                          return this;
                      }
-                 }));
+                 };
+                 futures.add(executor.submit(callable));
+                 if (jobs > 0 && futures.size() == jobs)
+                 {
+                     FBUtilities.waitOnFutures(futures);
+                     futures.clear();
+                 }
              }
- 
-             assert compacting.originals().isEmpty();
- 
              FBUtilities.waitOnFutures(futures);
++            assert compacting.originals().isEmpty();
 +            return AllSSTableOpStatus.SUCCESSFUL;
          }
          finally
          {
@@@ -327,9 -358,9 +341,9 @@@
          }
      }
  
-     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData)
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
 +    throws InterruptedException, ExecutionException
      {
 -        assert !cfs.isIndex();
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
              @Override
@@@ -343,29 -374,10 +357,29 @@@
              {
                  scrubOne(cfs, input, skipCorrupted, checkData);
              }
-         }, OperationType.SCRUB);
 -        }, jobs);
++        }, jobs, OperationType.SCRUB);
 +    }
 +
 +    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
 +    {
 +        assert !cfs.isIndex();
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
 +            {
 +                return input.originals();
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction input) throws IOException
 +            {
 +                verifyOne(cfs, input.onlyOne(), extendedVerify);
 +            }
-         }, OperationType.VERIFY);
++        }, 0, OperationType.VERIFY);
      }
  
-     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
      {
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
@@@ -394,10 -402,10 +408,10 @@@
                  task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                  task.execute(metrics);
              }
-         }, OperationType.UPGRADE_SSTABLES);
 -        }, jobs);
++        }, jobs, OperationType.UPGRADE_SSTABLES);
      }
  
-     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
      {
          assert !cfStore.isIndex();
          Keyspace keyspace = cfStore.keyspace;
@@@ -425,12 -428,12 +439,12 @@@
              }
  
              @Override
 -            public void execute(SSTableReader input) throws IOException
 +            public void execute(LifecycleTransaction txn) throws IOException
              {
                  CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
 -                doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
 +                doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
 -        }, jobs);
++        }, jobs, OperationType.CLEANUP);
      }
  
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 5d29a5a,507aedb..bca5996
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2555,7 -2385,12 +2555,12 @@@ public class StorageService extends Not
  
      public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+     }
+ 
+     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
 -        if (keyspaceName.equals(Keyspace.SYSTEM_KS))
 +        if (keyspaceName.equals(SystemKeyspace.NAME))
              throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
  
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
@@@ -2575,30 -2410,27 +2580,39 @@@
  
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+     }
+ 
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
          return status.statusCode;
      }
- 
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        {
 +            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.verify(extendedVerify);
 +            if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
 +                status = oneStatus;
 +        }
 +        return status.statusCode;
 +    }
 +
      public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+     }
+ 
+     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
          for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 7c5cd0a,d3a1725..761eed6
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -261,17 -272,11 +263,19 @@@ public interface StorageServiceMBean ex
       */
      @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
 +     * Verify (checksums of) the given keyspace.
 +     * If columnFamilies array is empty, all CFs are verified.
 +     *
 +     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
 +     */
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +
 +    /**
       * Rewrite all sstables to the latest version.
       * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 93804a8,ab08e9f..2f27cea
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -53,6 -62,8 +53,7 @@@ import javax.management.remote.JMXConne
  import javax.management.remote.JMXServiceURL;
  import javax.rmi.ssl.SslRMIClientSocketFactory;
  
 -import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStoreMBean;
  import org.apache.cassandra.db.HintedHandOffManager;
  import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@@ -230,60 -238,53 +231,69 @@@ public class NodeProbe implements AutoC
          jmxc.close();
      }
  
-     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
      }
  
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +    }
 +
-     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
      }
  
-     public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     private void checkJobs(PrintStream out, int jobs)
      {
-         if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+         if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+             out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+     }
+ 
+     public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
++        checkJobs(out, jobs);
+         if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+         checkJobs(out, jobs);
+         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +        }
 +    }
 +
 +    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
 +        {
 +            failed = true;
 +            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
 +
-     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+         checkJobs(out, jobs);
+         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index aa415b3,0000000..6c6676d
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@@ -1,56 -1,0 +1,62 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
++import io.airlift.command.Option;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "cleanup", description = "Triggers the immediate cleanup of keys no longer belonging to a node. By default, clean all keyspaces")
 +public class Cleanup extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to cleanup simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            if (SystemKeyspace.NAME.equals(keyspace))
 +                continue;
 +
 +            try
 +            {
-                 probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
++                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during cleanup", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 54f981e,0000000..dafe8d1
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -1,71 -1,0 +1,76 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
 +public class Scrub extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "disable_snapshot",
 +            name = {"-ns", "--no-snapshot"},
 +            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +    private boolean disableSnapshot = false;
 +
 +    @Option(title = "skip_corrupted",
 +            name = {"-s", "--skip-corrupted"},
 +            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +    private boolean skipCorrupted = false;
 +
 +    @Option(title = "no_validate",
 +                   name = {"-n", "--no-validate"},
 +                   description = "Do not validate columns using column validator")
 +    private boolean noValidation = false;
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
 +            } catch (IllegalArgumentException e)
 +            {
 +                throw e;
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during scrubbing", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
index 86a2cd5,0000000..596f353
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
@@@ -1,56 -1,0 +1,61 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "upgradesstables", description = "Rewrite sstables (for the requested tables) that are not on the current version (thus upgrading them to said current version)")
 +public class UpgradeSSTable extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
 +    private boolean includeAll = false;
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to upgrade simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
++                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during enabling auto-compaction", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index c0cde40,4efd082..4cca7ff
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -588,14 -499,14 +588,14 @@@ public class ScrubTes
      @Test
      public void testScrubColumnValidation() throws InterruptedException, RequestExecutionException, ExecutionException
      {
 -        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", ConsistencyLevel.ONE);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
  
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_static_columns");
  
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')", KEYSPACE));
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
          ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@@ -614,15 -525,15 +614,15 @@@
      @Test
      public void testColumnNameEqualToDefaultKeyAlias() throws ExecutionException, InterruptedException
      {
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 -        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("UUIDKeys");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_UUID);
  
 -        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "UUIDKeys");
 +        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF_UUID);
          cf.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
 -        Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
 +        Mutation mutation = new Mutation(KEYSPACE, ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
          mutation.applyUnsafe();
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          assertEquals(1, cfs.getSSTables().size());
      }
@@@ -634,19 -545,19 +634,19 @@@
      @Test
      public void testValidationCompactStorage() throws Exception
      {
 -        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", ConsistencyLevel.ONE);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", KEYSPACE), ConsistencyLevel.ONE);
  
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_dynamic_columns");
  
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')");
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')", KEYSPACE));
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')", KEYSPACE));
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')", KEYSPACE));
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, true, true);
+         CompactionManager.instance.performScrub(cfs, true, true, 2);
  
          // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
 -        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".test_compact_dynamic_columns", KEYSPACE));
          assertEquals(3, rs.size());
  
          Iterator<UntypedResultSet.Row> iter = rs.iterator();
@@@ -654,129 -565,4 +654,129 @@@
          assertEquals("bar", iter.next().getString("c"));
          assertEquals("boo", iter.next().getString("c"));
      }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
 +        // otherwise it uses LocalByPartitionerType
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
 +    }
 +
 +    /** The SecondaryIndex class is used for custom indexes so to avoid
 +     * making a public final field into a private field with getters
 +     * and setters, we resort to this hack in order to test it properly
 +     * since it can have two values which influence the scrubbing behavior.
 +     * @param comparator - the key comparator we want to test
 +     */
 +    private void setKeyComparator(AbstractType<?> comparator)
 +    {
 +        try
 +        {
 +            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
 +            keyComparator.setAccessible(true);
 +            int modifiers = keyComparator.getModifiers();
 +            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
 +            modifiers = modifiers & ~Modifier.FINAL;
 +            modifierField.setAccessible(true);
 +            modifierField.setInt(keyComparator, modifiers);
 +
 +            keyComparator.set(null, comparator);
 +        }
 +        catch (Exception ex)
 +        {
 +            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
 +            ex.printStackTrace();
 +        }
 +    }
 +
 +    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
 +            throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.instance.disableAutoCompaction();
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 +        cfs.clearUnsafe();
 +
 +        int numRows = 1000;
 +        long[] colValues = new long [numRows * 2]; // each row has two columns
 +        for (int i = 0; i < colValues.length; i+=2)
 +        {
 +            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
 +            colValues[i+1] = 3L; //other column
 +        }
 +        fillIndexCF(cfs, composite, colValues);
 +
 +        // check index
 +        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
 +        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +
 +        // scrub index
 +        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
 +        assertTrue(indexCfss.size() == 1);
 +        for(ColumnFamilyStore indexCfs : indexCfss)
 +        {
 +            for (int i = 0; i < scrubs.length; i++)
 +            {
 +                boolean failure = !scrubs[i];
 +                if (failure)
 +                { //make sure the next scrub fails
 +                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
 +                assertEquals(failure ?
 +                             CompactionManager.AllSSTableOpStatus.ABORTED :
 +                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
 +                                result);
 +            }
 +        }
 +
 +
 +        // check index is still working
 +        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------


[15/15] cassandra git commit: Merge branch 'cassandra-3.5' into trunk

Posted by ma...@apache.org.
Merge branch 'cassandra-3.5' into trunk


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

Branch: refs/heads/trunk
Commit: f04224707d29544228e791f0b532a9969da2820b
Parents: a0a9494 0a25085
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 11:10:28 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 11:10:28 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 20 ++++-----
 .../db/compaction/CompactionManager.java        | 44 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 30 ++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  8 ++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 38 ++++++++++-------
 .../cassandra/tools/nodetool/Cleanup.java       |  9 +++-
 .../tools/nodetool/RelocateSSTables.java        |  8 +++-
 .../apache/cassandra/tools/nodetool/Scrub.java  | 13 ++++--
 .../tools/nodetool/UpgradeSSTable.java          | 10 ++++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 16 +++----
 12 files changed, 141 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0422470/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------


[11/15] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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

Branch: refs/heads/cassandra-3.5
Commit: 0c91977da68f054cf0ddf996b49423d736b42006
Parents: 3efc609 897ffe8
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 11:00:13 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 11:00:13 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 16 ++++----
 .../db/compaction/CompactionManager.java        | 40 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 23 +++++++++--
 .../cassandra/service/StorageServiceMBean.java  |  6 +++
 .../org/apache/cassandra/tools/NodeProbe.java   | 34 +++++++++++------
 .../cassandra/tools/nodetool/Cleanup.java       |  9 ++++-
 .../apache/cassandra/tools/nodetool/Scrub.java  | 13 +++++--
 .../tools/nodetool/UpgradeSSTable.java          | 10 ++++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 16 ++++----
 11 files changed, 120 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index fe19df7,098d062..fdc873e
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -20,36 -8,13 +20,38 @@@ Merged from 2.2
   * Add missing newline at end of bin/cqlsh (CASSANDRA-11325)
   * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
   * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 - * Fix filtering on non-primary key columns for thrift static column families
 -   (CASSANDRA-6377)
   * Only log yaml config once, at startup (CASSANDRA-11217)
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 +Merged from 2.1:
++ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
++   many threads to use (CASSANDRA-11179)
 + * Backport CASSANDRA-10679 (CASSANDRA-9598)
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
 + * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 +
 +3.0.4
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * MV should only query complex columns included in the view (CASSANDRA-11069)
 + * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
 + * Add sstabledump tool (CASSANDRA-7464)
 + * Introduce backpressure for hints (CASSANDRA-10972)
 + * Fix ClusteringPrefix not being able to read tombstone range boundaries (CASSANDRA-11158)
 + * Prevent logging in sandboxed state (CASSANDRA-11033)
 + * Disallow drop/alter operations of UDTs used by UDAs (CASSANDRA-10721)
 + * Add query time validation method on Index (CASSANDRA-11043)
 + * Avoid potential AssertionError in mixed version cluster (CASSANDRA-11128)
 + * Properly handle hinted handoff after topology changes (CASSANDRA-5902)
 + * AssertionError when listing sstable files on inconsistent disk state (CASSANDRA-11156)
 + * Fix wrong rack counting and invalid conditions check for TokenAllocation
 +   (CASSANDRA-11139)
 + * Avoid creating empty hint files (CASSANDRA-11090)
 + * Fix leak detection strong reference loop using weak reference (CASSANDRA-11120)
 + * Configurie BatchlogManager to stop delayed tasks on shutdown (CASSANDRA-11062)
 + * Hadoop integration is incompatible with Cassandra Driver 3.0.0 (CASSANDRA-11001)
 + * Add dropped_columns to the list of schema table so it gets handled
 +   properly (CASSANDRA-11050)
 + * Fix NPE when using forceRepairRangeAsync without DC (CASSANDRA-11239)
 +Merged from 2.2:
   * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
   * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
   * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 7c46fcb,ca02747..f339660
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -434,10 -441,10 +448,10 @@@ public class CompactionManager implemen
              @Override
              public void execute(LifecycleTransaction txn) throws IOException
              {
 -                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
 +                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges, FBUtilities.nowInSeconds());
                  doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
+         }, jobs, OperationType.CLEANUP);
      }
  
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 3a62063,bca5996..75573ac
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2633,32 -2553,42 +2633,42 @@@ public class StorageService extends Not
          return Gossiper.instance.getCurrentGenerationNumber(FBUtilities.getBroadcastAddress());
      }
  
 -    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
++        return forceKeyspaceCleanup(0, keyspaceName, tables);
+     }
+ 
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
 -        if (keyspaceName.equals(SystemKeyspace.NAME))
 +        if (Schema.isSystemKeyspace(keyspaceName))
              throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
  
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup(jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
          return status.statusCode;
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, tables);
 -        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, tables);
+     }
+ 
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
@@@ -2677,12 -2606,17 +2687,17 @@@
          return status.statusCode;
      }
  
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
++        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 0, tableNames);
+     }
+ 
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, tableNames))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 049b4f9,761eed6..5b7331d
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -251,7 -251,9 +251,9 @@@ public interface StorageServiceMBean ex
      /**
       * Trigger a cleanup of keys on a single keyspace
       */
+     @Deprecated
 -    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException;
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@@ -260,8 -262,10 +262,10 @@@
       * Scrubbed CFs will be snapshotted first, if disableSnapshot is false
       */
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Verify (checksums of) the given keyspace.
@@@ -275,7 -279,9 +279,9 @@@
       * Rewrite all sstables to the latest version.
       * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
       */
+     @Deprecated
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException;
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Flush all memtables for the given column families, or all columnfamilies for the given keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index cdce497,2f27cea..a9d71d8
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -53,9 -53,9 +53,10 @@@ import javax.management.remote.JMXConne
  import javax.management.remote.JMXServiceURL;
  import javax.rmi.ssl.SslRMIClientSocketFactory;
  
 +import org.apache.cassandra.batchlog.BatchlogManager;
 +import org.apache.cassandra.batchlog.BatchlogManagerMBean;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 -import org.apache.cassandra.db.HintedHandOffManager;
  import org.apache.cassandra.db.HintedHandOffManagerMBean;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.compaction.CompactionManagerMBean;
@@@ -236,38 -231,46 +237,46 @@@ public class NodeProbe implements AutoC
          jmxc.close();
      }
  
-     public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.forceKeyspaceCleanup(keyspaceName, tables);
 -        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
++        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, tables);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, tables);
 -        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
++        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables);
      }
  
 -    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +        return ssProxy.verify(extendedVerify, keyspaceName, tableNames);
      }
  
-     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, tableNames);
 -        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
++        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames);
      }
  
-     public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     private void checkJobs(PrintStream out, int jobs)
      {
-         if (forceKeyspaceCleanup(keyspaceName, tableNames) != 0)
+         if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+             out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+     }
+ 
 -    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     {
+         checkJobs(out, jobs);
 -        if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
++        if (forceKeyspaceCleanup(jobs, keyspaceName, tableNames) != 0)
          {
              failed = true;
              out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, tables) != 0)
+         checkJobs(out, jobs);
 -        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
++        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables) != 0)
          {
              failed = true;
              out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
@@@ -284,9 -287,10 +293,10 @@@
      }
  
  
-     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
 -    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
-         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, tableNames) != 0)
+         checkJobs(out, jobs);
 -        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
++        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames) != 0)
          {
              failed = true;
              out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index cbfd4d6,6c6676d..a7507b1
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@@ -23,8 -23,8 +23,8 @@@ import io.airlift.command.Command
  import java.util.ArrayList;
  import java.util.List;
  
 +import org.apache.cassandra.config.Schema;
- import org.apache.cassandra.db.SystemKeyspace;
+ import io.airlift.command.Option;
 -import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.tools.NodeProbe;
  import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
  
@@@ -47,9 -52,8 +52,9 @@@ public class Cleanup extends NodeToolCm
  
              try
              {
-                 probe.forceKeyspaceCleanup(System.out, keyspace, tableNames);
 -                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
 -            } catch (Exception e)
++                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, tableNames);
 +            }
 +            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during cleanup", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index f92ae0d,dafe8d1..2345a85
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -58,11 -63,11 +63,13 @@@ public class Scrub extends NodeToolCm
          {
              try
              {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, tableNames);
 -                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
--            } catch (IllegalArgumentException e)
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, tableNames);
++            }
++            catch (IllegalArgumentException e)
              {
                  throw e;
--            } catch (Exception e)
++            }
++            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during scrubbing", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
index a775878,596f353..82866e0
--- a/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
@@@ -46,8 -51,8 +51,9 @@@ public class UpgradeSSTable extends Nod
          {
              try
              {
-                 probe.upgradeSSTables(System.out, keyspace, !includeAll, tableNames);
 -                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
--            } catch (Exception e)
++                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, tableNames);
++            }
++            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during enabling auto-compaction", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/CleanupTest.java
index 167f3b0,5777af4..b4ffe57
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@@ -92,11 -94,11 +92,11 @@@ public class CleanupTes
          // record max timestamps of the sstables pre-cleanup
          List<Long> expectedMaxTimestamps = getMaxTimestampList(cfs);
  
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(LOOPS, rows.size());
 +        iter = Util.getRangeSlice(cfs);
 +        assertEquals(LOOPS, Iterators.size(iter));
  
          // with one token in the ring, owned by the local node, cleanup should be a no-op
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
          // ensure max timestamp of the sstables are retained post-cleanup
          assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
@@@ -137,16 -143,18 +137,16 @@@
          tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
          tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
  
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
          // row data should be gone
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
  
          // not only should it be gone but there should be no data on disk, not even tombstones
 -        assert cfs.getSSTables().isEmpty();
 +        assert cfs.getLiveSSTables().isEmpty();
  
          // 2ary indexes should result in no results, too (although tombstones won't be gone until compacted)
 -        rows = cfs.search(range, clause, filter, Integer.MAX_VALUE);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).filterOn("birthdate", Operator.EQ, VALUE).build()).size());
      }
  
      @Test
@@@ -168,9 -180,10 +168,9 @@@
          tk2[0] = 1;
          tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
          tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
      }
  
      @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 6dbbb1b,4cca7ff..936ccd8
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -110,14 -127,18 +110,14 @@@ public class ScrubTes
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 1);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
      }
  
      @Test
@@@ -235,17 -260,21 +235,17 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 4);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(4, rows.size());
 +        assertOrderedAll(cfs, 4);
  
 -        SSTableReader sstable = cfs.getSSTables().iterator().next();
 +        SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
          overrideWithGarbage(sstable, 0, 2);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(4, rows.size());
 +        assertOrderedAll(cfs, 4);
      }
  
      @Test
@@@ -271,14 -318,18 +271,14 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 10);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
      }
  
      @Test
@@@ -289,17 -340,21 +289,17 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 10);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
  
 -        for (SSTableReader sstable : cfs.getSSTables())
 +        for (SSTableReader sstable : cfs.getLiveSSTables())
              new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)).delete();
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
      }
  
      @Test
@@@ -504,11 -599,13 +504,11 @@@
  
          QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
          ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
 -        Mutation mutation = new Mutation("Keyspace1", UTF8Type.instance.decompose("key"));
 -        CellNameType ct = cfs2.getComparator();
 -        mutation.add("test_scrub_validation", ct.makeCellName("b"), LongType.instance.decompose(1L), System.currentTimeMillis());
 -        mutation.apply();
 +
 +        new Mutation(UpdateBuilder.create(cfs2.metadata, "key").newRow().add("b", LongType.instance.decompose(1L)).build()).apply();
          cfs2.forceBlockingFlush();
  
-         CompactionManager.instance.performScrub(cfs2, false, false);
+         CompactionManager.instance.performScrub(cfs2, false, false, 2);
      }
  
      /**
@@@ -614,9 -763,9 +614,9 @@@
                  boolean failure = !scrubs[i];
                  if (failure)
                  { //make sure the next scrub fails
 -                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                    overrideWithGarbage(indexCfs.getLiveSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
                  }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true);
+                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
                  assertEquals(failure ?
                               CompactionManager.AllSSTableOpStatus.ABORTED :
                               CompactionManager.AllSSTableOpStatus.SUCCESSFUL,


[13/15] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.5

Posted by ma...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.5


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

Branch: refs/heads/cassandra-3.5
Commit: 0a2508544069f035dc43a81b8700c0b7bf2d28e3
Parents: c7ef7c9 0c91977
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 11:05:46 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 11:05:46 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 20 ++++-----
 .../db/compaction/CompactionManager.java        | 44 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 30 ++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  8 ++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 38 ++++++++++-------
 .../cassandra/tools/nodetool/Cleanup.java       |  9 +++-
 .../tools/nodetool/RelocateSSTables.java        |  8 +++-
 .../apache/cassandra/tools/nodetool/Scrub.java  | 13 ++++--
 .../tools/nodetool/UpgradeSSTable.java          | 10 ++++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 16 +++----
 12 files changed, 141 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 2907df9,fdc873e..3e69959
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -25,43 -23,14 +25,45 @@@ Merged from 2.2
   * Only log yaml config once, at startup (CASSANDRA-11217)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
  Merged from 2.1:
+  * Add a -j parameter to scrub/cleanup/upgradesstables to state how
+    many threads to use (CASSANDRA-11179)
 - * Backport CASSANDRA-10679 (CASSANDRA-9598)
 - * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
   * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  
 -3.0.4
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 +
 +3.4
 + * (cqlsh) add cqlshrc option to always connect using ssl (CASSANDRA-10458)
 + * Cleanup a few resource warnings (CASSANDRA-11085)
 + * Allow custom tracing implementations (CASSANDRA-10392)
 + * Extract LoaderOptions to be able to be used from outside (CASSANDRA-10637)
 + * fix OnDiskIndexTest to properly treat empty ranges (CASSANDRA-11205)
 + * fix TrackerTest to handle new notifications (CASSANDRA-11178)
 + * add SASI validation for partitioner and complex columns (CASSANDRA-11169)
 + * Add caching of encrypted credentials in PasswordAuthenticator (CASSANDRA-7715)
 + * fix SASI memtable switching on flush (CASSANDRA-11159)
 + * Remove duplicate offline compaction tracking (CASSANDRA-11148)
 + * fix EQ semantics of analyzed SASI indexes (CASSANDRA-11130)
 + * Support long name output for nodetool commands (CASSANDRA-7950)
 + * Encrypted hints (CASSANDRA-11040)
 + * SASI index options validation (CASSANDRA-11136)
 + * Optimize disk seek using min/max column name meta data when the LIMIT clause is used
 +   (CASSANDRA-8180)
 + * Add LIKE support to CQL3 (CASSANDRA-11067)
 + * Generic Java UDF types (CASSANDRA-10819)
 + * cqlsh: Include sub-second precision in timestamps by default (CASSANDRA-10428)
 + * Set javac encoding to utf-8 (CASSANDRA-11077)
 + * Integrate SASI index into Cassandra (CASSANDRA-10661)
 + * Add --skip-flush option to nodetool snapshot
 + * Skip values for non-queried columns (CASSANDRA-10657)
 + * Add support for secondary indexes on static columns (CASSANDRA-8103)
 + * CommitLogUpgradeTestMaker creates broken commit logs (CASSANDRA-11051)
 + * Add metric for number of dropped mutations (CASSANDRA-10866)
 + * Simplify row cache invalidation code (CASSANDRA-10396)
 + * Support user-defined compaction through nodetool (CASSANDRA-10660)
 + * Stripe view locks by key and table ID to reduce contention (CASSANDRA-10981)
 + * Add nodetool gettimeout and settimeout commands (CASSANDRA-10953)
 + * Add 3.0 metadata to sstablemetadata output (CASSANDRA-10838)
 +Merged from 3.0:
   * MV should only query complex columns included in the view (CASSANDRA-11069)
   * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
   * Add sstabledump tool (CASSANDRA-7464)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index fbfb7ee,a1db5b5..f193c4d
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1436,16 -1363,11 +1436,16 @@@ public class ColumnFamilyStore implemen
          return CompactionManager.instance.performVerify(ColumnFamilyStore.this, extendedVerify);
      }
  
-     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus relocateSSTables() throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus relocateSSTables(int jobs) throws ExecutionException, InterruptedException
 +    {
-         return CompactionManager.instance.relocateSSTables(this);
++        return CompactionManager.instance.relocateSSTables(this, jobs);
 +    }
 +
      public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
      {
          assert !sstables.isEmpty();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 627ae32,f339660..f9b0997
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -438,80 -451,9 +452,80 @@@ public class CompactionManager implemen
                  CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges, FBUtilities.nowInSeconds());
                  doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
+         }, jobs, OperationType.CLEANUP);
      }
  
-     public AllSSTableOpStatus relocateSSTables(final ColumnFamilyStore cfs) throws ExecutionException, InterruptedException
++    public AllSSTableOpStatus relocateSSTables(final ColumnFamilyStore cfs, int jobs) throws ExecutionException, InterruptedException
 +    {
 +        if (!cfs.getPartitioner().splitter().isPresent())
 +        {
 +            logger.info("Partitioner does not support splitting");
 +            return AllSSTableOpStatus.ABORTED;
 +        }
 +        final Collection<Range<Token>> r = StorageService.instance.getLocalRanges(cfs.keyspace.getName());
 +
 +        if (r.isEmpty())
 +        {
 +            logger.info("Relocate cannot run before a node has joined the ring");
 +            return AllSSTableOpStatus.ABORTED;
 +        }
 +
 +        final List<Range<Token>> localRanges = Range.sort(r);
 +        final Directories.DataDirectory[] locations = cfs.getDirectories().getWriteableLocations();
 +        final List<PartitionPosition> diskBoundaries = StorageService.getDiskBoundaries(localRanges, cfs.getPartitioner(), locations);
 +
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction transaction)
 +            {
 +                Set<SSTableReader> originals = Sets.newHashSet(transaction.originals());
 +                Set<SSTableReader> needsRelocation = originals.stream().filter(s -> !inCorrectLocation(s)).collect(Collectors.toSet());
 +                transaction.cancel(Sets.difference(originals, needsRelocation));
 +
 +                Map<Integer, List<SSTableReader>> groupedByDisk = needsRelocation.stream().collect(Collectors.groupingBy((s) ->
 +                        CompactionStrategyManager.getCompactionStrategyIndex(cfs, cfs.getDirectories(), s)));
 +
 +                int maxSize = 0;
 +                for (List<SSTableReader> diskSSTables : groupedByDisk.values())
 +                    maxSize = Math.max(maxSize, diskSSTables.size());
 +
 +                List<SSTableReader> mixedSSTables = new ArrayList<>();
 +
 +                for (int i = 0; i < maxSize; i++)
 +                    for (List<SSTableReader> diskSSTables : groupedByDisk.values())
 +                        if (i < diskSSTables.size())
 +                            mixedSSTables.add(diskSSTables.get(i));
 +
 +                return mixedSSTables;
 +            }
 +
 +            private boolean inCorrectLocation(SSTableReader sstable)
 +            {
 +                if (!cfs.getPartitioner().splitter().isPresent())
 +                    return true;
 +                int directoryIndex = CompactionStrategyManager.getCompactionStrategyIndex(cfs, cfs.getDirectories(), sstable);
 +                Directories.DataDirectory[] locations = cfs.getDirectories().getWriteableLocations();
 +
 +                Directories.DataDirectory location = locations[directoryIndex];
 +                PartitionPosition diskLast = diskBoundaries.get(directoryIndex);
 +                // the location we get from directoryIndex is based on the first key in the sstable
 +                // now we need to make sure the last key is less than the boundary as well:
 +                return sstable.descriptor.directory.getAbsolutePath().startsWith(location.location.getAbsolutePath()) && sstable.last.compareTo(diskLast) <= 0;
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction txn) throws IOException
 +            {
 +                logger.debug("Relocating {}", txn.originals());
 +                AbstractCompactionTask task = cfs.getCompactionStrategyManager().getCompactionTask(txn, NO_GC, Long.MAX_VALUE);
 +                task.setUserDefined(true);
 +                task.setCompactionType(OperationType.RELOCATE);
 +                task.execute(metrics);
 +            }
-         }, OperationType.RELOCATE);
++        }, jobs, OperationType.RELOCATE);
 +    }
 +
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,
                                            final Collection<Range<Token>> ranges,
                                            final Refs<SSTableReader> sstables,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 22d67e5,75573ac..52dcb97
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2789,59 -2712,6 +2804,64 @@@ public class StorageService extends Not
          }
      }
  
 +    public int relocateSSTables(String keyspaceName, String ... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
++        return relocateSSTables(0, keyspaceName, columnFamilies);
++    }
++
++    public int relocateSSTables(int jobs, String keyspaceName, String ... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    {
 +        CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        for (ColumnFamilyStore cfs : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfs.relocateSSTables();
++            CompactionManager.AllSSTableOpStatus oneStatus = cfs.relocateSSTables(jobs);
 +            if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
 +                status = oneStatus;
 +        }
 +        return status.statusCode;
 +    }
 +
 +    /**
 +     * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
 +     *
 +     * @param tag
 +     *            the tag given to the snapshot; may not be null or empty
 +     * @param options
 +     *            Map of options (skipFlush is the only supported option for now)
 +     * @param entities
 +     *            list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,...
 +     */
 +    @Override
 +    public void takeSnapshot(String tag, Map<String, String> options, String... entities) throws IOException
 +    {
 +        boolean skipFlush = Boolean.parseBoolean(options.getOrDefault("skipFlush", "false"));
 +
 +        if (entities != null && entities.length > 0 && entities[0].contains("."))
 +        {
 +            takeMultipleTableSnapshot(tag, skipFlush, entities);
 +        }
 +        else
 +        {
 +            takeSnapshot(tag, skipFlush, entities);
 +        }
 +    }
 +
 +    /**
 +     * Takes the snapshot of a specific table. A snapshot name must be
 +     * specified.
 +     *
 +     * @param keyspaceName
 +     *            the keyspace which holds the specified table
 +     * @param tableName
 +     *            the table to snapshot
 +     * @param tag
 +     *            the tag given to the snapshot; may not be null or empty
 +     */
 +    public void takeTableSnapshot(String keyspaceName, String tableName, String tag)
 +            throws IOException {
 +        takeMultipleTableSnapshot(tag, false, keyspaceName + "." + tableName);
 +    }
 +
      /**
       * Takes the snapshot for the given keyspaces. A snapshot name must be specified.
       *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index dc12253,5b7331d..2238f33
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -251,7 -248,6 +251,9 @@@ public interface StorageServiceMBean ex
       */
      public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
  
++    @Deprecated
 +    public int relocateSSTables(String keyspace, String ... cfnames) throws IOException, ExecutionException, InterruptedException;
++    public int relocateSSTables(int jobs, String keyspace, String ... cfnames) throws IOException, ExecutionException, InterruptedException;
      /**
       * Trigger a cleanup of keys on a single keyspace
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 405f70c,a9d71d8..7d15926
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -304,11 -309,6 +314,11 @@@ public class NodeProbe implements AutoC
          ssProxy.forceKeyspaceCompaction(splitOutput, keyspaceName, tableNames);
      }
  
-     public void relocateSSTables(String keyspace, String[] cfnames) throws IOException, ExecutionException, InterruptedException
++    public void relocateSSTables(int jobs, String keyspace, String[] cfnames) throws IOException, ExecutionException, InterruptedException
 +    {
-         ssProxy.relocateSSTables(keyspace, cfnames);
++        ssProxy.relocateSSTables(jobs, keyspace, cfnames);
 +    }
 +
      public void forceKeyspaceFlush(String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
          ssProxy.forceKeyspaceFlush(keyspaceName, tableNames);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/src/java/org/apache/cassandra/tools/nodetool/RelocateSSTables.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/RelocateSSTables.java
index 8522bc4,0000000..7c3066c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/RelocateSSTables.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/RelocateSSTables.java
@@@ -1,49 -1,0 +1,55 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
++import io.airlift.command.Option;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool;
 +
 +@Command(name = "relocatesstables", description = "Relocates sstables to the correct disk")
 +public class RelocateSSTables extends NodeTool.NodeToolCmd
 +{
 +    @Arguments(usage = "<keyspace> <table>", description = "The keyspace and table name")
 +    private List<String> args = new ArrayList<>();
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to relocate simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalTables(args);
 +        try
 +        {
 +            for (String keyspace : keyspaces)
-                 probe.relocateSSTables(keyspace, cfnames);
++                probe.relocateSSTables(jobs, keyspace, cfnames);
 +        }
 +        catch (Exception e)
 +        {
 +            throw new RuntimeException("Got error while relocating", e);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a250854/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------


[06/15] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

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


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

Branch: refs/heads/trunk
Commit: 897ffe87e41ab128c9e8969d535cb2a706baf563
Parents: 6c1ef2b 8b8a3f5
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 10:54:45 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:54:45 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 16 ++++----
 .../db/compaction/CompactionManager.java        | 40 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 24 +++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 +++
 .../org/apache/cassandra/tools/NodeProbe.java   | 34 +++++++++++------
 .../cassandra/tools/nodetool/Cleanup.java       |  8 +++-
 .../apache/cassandra/tools/nodetool/Scrub.java  |  7 +++-
 .../tools/nodetool/UpgradeSSTable.java          |  7 +++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 20 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 12 files changed, 117 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 91dc588,7794d4f..098d062
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,39 -1,4 +1,41 @@@
 -2.1.14
 +2.2.6
 + * cqlsh: COPY FROM should check that explicit column names are valid (CASSANDRA-11333)
 + * Add -Dcassandra.start_gossip startup option (CASSANDRA-10809)
 + * Fix UTF8Validator.validate() for modified UTF-8 (CASSANDRA-10748)
 + * Clarify that now() function is calculated on the coordinator node in CQL documentation (CASSANDRA-10900)
 + * Fix bloom filter sizing with LCS (CASSANDRA-11344)
 + * (cqlsh) Fix error when result is 0 rows with EXPAND ON (CASSANDRA-11092)
 + * Fix intra-node serialization issue for multicolumn-restrictions (CASSANDRA-11196)
 + * Non-obsoleting compaction operations over compressed files can impose rate limit on normal reads (CASSANDRA-11301)
 + * Add missing newline at end of bin/cqlsh (CASSANDRA-11325)
 + * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 + * Fix filtering on non-primary key columns for thrift static column families
 +   (CASSANDRA-6377)
 + * Only log yaml config once, at startup (CASSANDRA-11217)
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 + * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
 + * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
 + * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
 + * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
 + * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
 + * Protect from keyspace dropped during repair (CASSANDRA-11065)
 + * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
 + * Better error message for cleanup (CASSANDRA-10991)
 + * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
 + * Use cloned TokenMetadata in size estimates to avoid race against membership check
 +   (CASSANDRA-10736)
 + * Always persist upsampled index summaries (CASSANDRA-10512)
 + * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
 + * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
 + * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
 + * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
 + * Fix paging on DISTINCT queries repeats result when first row in partition changes
 +   (CASSANDRA-10010)
 +Merged from 2.1:
++ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
++   many threads to use (CASSANDRA-11179)
   * Backport CASSANDRA-10679 (CASSANDRA-9598)
   * Don't do defragmentation if reading from repaired sstables (CASSANDRA-10342)
   * Fix streaming_socket_timeout_in_ms not enforced (CASSANDRA-11286)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index da4a84a,3d66d3a..09f58ac
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1499,74 -1478,22 +1499,74 @@@ public class ColumnFamilyStore implemen
          return maxFile;
      }
  
-     public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData);
++        return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 +    }
 +
 +    @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 +    {
          // skip snapshot creation during scrub, SEE JIRA 5891
          if(!disableSnapshot)
              snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
 -        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 +
 +        try
 +        {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 +        }
 +        catch(Throwable t)
 +        {
 +            if (!rebuildOnFailedScrub(t))
 +                throw t;
 +
 +            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        }
 +    }
 +
 +    /**
 +     * CASSANDRA-5174 : For an index cfs we may be able to discard everything and just rebuild
 +     * the index when a scrub fails.
 +     *
 +     * @return true if we are an index cfs and we successfully rebuilt the index
 +     */
 +    public boolean rebuildOnFailedScrub(Throwable failure)
 +    {
 +        if (!isIndex())
 +            return false;
 +
 +        SecondaryIndex index = null;
 +        if (metadata.cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR))
 +        {
 +            String[] parts = metadata.cfName.split("\\" + Directories.SECONDARY_INDEX_NAME_SEPARATOR, 2);
 +            ColumnFamilyStore parentCfs = keyspace.getColumnFamilyStore(parts[0]);
 +            index = parentCfs.indexManager.getIndexByName(metadata.cfName);
 +            assert index != null;
 +        }
 +
 +        if (index == null)
 +            return false;
 +
 +        truncateBlocking();
 +
 +        logger.warn("Rebuilding index for {} because of <{}>", name, failure.getMessage());
 +        index.getBaseCfs().rebuildSecondaryIndex(index.getIndexName());
 +        return true;
 +    }
 +
 +    public CompactionManager.AllSSTableOpStatus verify(boolean extendedVerify) throws ExecutionException, InterruptedException
 +    {
 +        return CompactionManager.instance.performVerify(ColumnFamilyStore.this, extendedVerify);
      }
  
-     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
      {
-         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+         return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
      }
  
      public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index b015bcd,e382cab..ca02747
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -263,21 -271,37 +263,31 @@@ public class CompactionManager implemen
          }
      }
  
+     /**
+      * Run an operation over all sstables using jobs threads
+      *
+      * @param cfs the column family store to run the operation on
+      * @param operation the operation to run
+      * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+      * @return status of the operation
+      * @throws ExecutionException
+      * @throws InterruptedException
+      */
 -    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
 +    @SuppressWarnings("resource")
-     private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, OperationType operationType) throws ExecutionException, InterruptedException
++    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs, OperationType operationType) throws ExecutionException, InterruptedException
      {
 -        Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
 -        if (compactingSSTables == null)
 +        List<LifecycleTransaction> transactions = new ArrayList<>();
 +        try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType))
          {
 -            logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
 -            return AllSSTableOpStatus.ABORTED;
 -        }
 -        if (Iterables.isEmpty(compactingSSTables))
 -        {
 -            logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
 -            return AllSSTableOpStatus.SUCCESSFUL;
 -        }
 -        Set<SSTableReader> sstables = Sets.newHashSet(operation.filterSSTables(compactingSSTables));
 -        Set<SSTableReader> filteredAway = Sets.difference(Sets.newHashSet(compactingSSTables), sstables);
 -        cfs.getDataTracker().unmarkCompacting(filteredAway);
 -        final Set<SSTableReader> finished = Sets.newConcurrentHashSet();
 +            Iterable<SSTableReader> sstables = compacting != null ? Lists.newArrayList(operation.filterSSTables(compacting)) : Collections.<SSTableReader>emptyList();
 +            if (Iterables.isEmpty(sstables))
 +            {
 +                logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
 +                return AllSSTableOpStatus.SUCCESSFUL;
 +            }
 +
 +            List<Future<Object>> futures = new ArrayList<>();
  
 -        List<Future<Object>> futures = new ArrayList<>();
 -        try
 -        {
              for (final SSTableReader sstable : sstables)
              {
                  if (executor.isShutdown())
@@@ -286,23 -310,31 +296,27 @@@
                      return AllSSTableOpStatus.ABORTED;
                  }
  
 +                final LifecycleTransaction txn = compacting.split(singleton(sstable));
 +                transactions.add(txn);
-                 futures.add(executor.submit(new Callable<Object>()
+                 Callable<Object> callable = new Callable<Object>()
                  {
                      @Override
                      public Object call() throws Exception
                      {
 -                        try
 -                        {
 -                            operation.execute(sstable);
 -                        }
 -                        finally
 -                        {
 -                            cfs.getDataTracker().unmarkCompacting(Collections.singleton(sstable));
 -                            finished.add(sstable);
 -                        }
 +                        operation.execute(txn);
                          return this;
                      }
-                 }));
+                 };
+                 futures.add(executor.submit(callable));
+                 if (jobs > 0 && futures.size() == jobs)
+                 {
+                     FBUtilities.waitOnFutures(futures);
+                     futures.clear();
+                 }
              }
- 
-             assert compacting.originals().isEmpty();
- 
              FBUtilities.waitOnFutures(futures);
++            assert compacting.originals().isEmpty();
 +            return AllSSTableOpStatus.SUCCESSFUL;
          }
          finally
          {
@@@ -327,9 -358,9 +341,9 @@@
          }
      }
  
-     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData)
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
 +    throws InterruptedException, ExecutionException
      {
 -        assert !cfs.isIndex();
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
              @Override
@@@ -343,29 -374,10 +357,29 @@@
              {
                  scrubOne(cfs, input, skipCorrupted, checkData);
              }
-         }, OperationType.SCRUB);
 -        }, jobs);
++        }, jobs, OperationType.SCRUB);
 +    }
 +
 +    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
 +    {
 +        assert !cfs.isIndex();
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
 +            {
 +                return input.originals();
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction input) throws IOException
 +            {
 +                verifyOne(cfs, input.onlyOne(), extendedVerify);
 +            }
-         }, OperationType.VERIFY);
++        }, 0, OperationType.VERIFY);
      }
  
-     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
      {
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
@@@ -394,10 -402,10 +408,10 @@@
                  task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                  task.execute(metrics);
              }
-         }, OperationType.UPGRADE_SSTABLES);
 -        }, jobs);
++        }, jobs, OperationType.UPGRADE_SSTABLES);
      }
  
-     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
      {
          assert !cfStore.isIndex();
          Keyspace keyspace = cfStore.keyspace;
@@@ -425,12 -428,12 +439,12 @@@
              }
  
              @Override
 -            public void execute(SSTableReader input) throws IOException
 +            public void execute(LifecycleTransaction txn) throws IOException
              {
                  CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
 -                doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
 +                doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
 -        }, jobs);
++        }, jobs, OperationType.CLEANUP);
      }
  
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 5d29a5a,507aedb..bca5996
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2555,7 -2385,12 +2555,12 @@@ public class StorageService extends Not
  
      public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+     }
+ 
+     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
 -        if (keyspaceName.equals(Keyspace.SYSTEM_KS))
 +        if (keyspaceName.equals(SystemKeyspace.NAME))
              throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
  
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
@@@ -2575,30 -2410,27 +2580,39 @@@
  
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+     }
+ 
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
          return status.statusCode;
      }
- 
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        {
 +            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.verify(extendedVerify);
 +            if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
 +                status = oneStatus;
 +        }
 +        return status.statusCode;
 +    }
 +
      public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+     }
+ 
+     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
          for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 7c5cd0a,d3a1725..761eed6
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -261,17 -272,11 +263,19 @@@ public interface StorageServiceMBean ex
       */
      @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
 +     * Verify (checksums of) the given keyspace.
 +     * If columnFamilies array is empty, all CFs are verified.
 +     *
 +     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
 +     */
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +
 +    /**
       * Rewrite all sstables to the latest version.
       * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 93804a8,ab08e9f..2f27cea
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -53,6 -62,8 +53,7 @@@ import javax.management.remote.JMXConne
  import javax.management.remote.JMXServiceURL;
  import javax.rmi.ssl.SslRMIClientSocketFactory;
  
 -import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStoreMBean;
  import org.apache.cassandra.db.HintedHandOffManager;
  import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@@ -230,60 -238,53 +231,69 @@@ public class NodeProbe implements AutoC
          jmxc.close();
      }
  
-     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
      }
  
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +    }
 +
-     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
      }
  
-     public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     private void checkJobs(PrintStream out, int jobs)
      {
-         if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+         if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+             out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+     }
+ 
+     public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
++        checkJobs(out, jobs);
+         if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+         checkJobs(out, jobs);
+         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +        }
 +    }
 +
 +    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
 +        {
 +            failed = true;
 +            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
 +
-     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+         checkJobs(out, jobs);
+         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index aa415b3,0000000..6c6676d
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@@ -1,56 -1,0 +1,62 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
++import io.airlift.command.Option;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "cleanup", description = "Triggers the immediate cleanup of keys no longer belonging to a node. By default, clean all keyspaces")
 +public class Cleanup extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to cleanup simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            if (SystemKeyspace.NAME.equals(keyspace))
 +                continue;
 +
 +            try
 +            {
-                 probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
++                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during cleanup", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 54f981e,0000000..dafe8d1
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -1,71 -1,0 +1,76 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
 +public class Scrub extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "disable_snapshot",
 +            name = {"-ns", "--no-snapshot"},
 +            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +    private boolean disableSnapshot = false;
 +
 +    @Option(title = "skip_corrupted",
 +            name = {"-s", "--skip-corrupted"},
 +            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +    private boolean skipCorrupted = false;
 +
 +    @Option(title = "no_validate",
 +                   name = {"-n", "--no-validate"},
 +                   description = "Do not validate columns using column validator")
 +    private boolean noValidation = false;
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
 +            } catch (IllegalArgumentException e)
 +            {
 +                throw e;
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during scrubbing", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
index 86a2cd5,0000000..596f353
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
@@@ -1,56 -1,0 +1,61 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "upgradesstables", description = "Rewrite sstables (for the requested tables) that are not on the current version (thus upgrading them to said current version)")
 +public class UpgradeSSTable extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
 +    private boolean includeAll = false;
 +
++    @Option(title = "jobs",
++            name = {"-j", "--jobs"},
++            description = "Number of sstables to upgrade simultanously, set to 0 to use all available compaction threads")
++    private int jobs = 2;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
++                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during enabling auto-compaction", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index c0cde40,4efd082..4cca7ff
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -588,14 -499,14 +588,14 @@@ public class ScrubTes
      @Test
      public void testScrubColumnValidation() throws InterruptedException, RequestExecutionException, ExecutionException
      {
 -        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", ConsistencyLevel.ONE);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
  
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_static_columns");
  
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')", KEYSPACE));
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
          ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@@ -614,15 -525,15 +614,15 @@@
      @Test
      public void testColumnNameEqualToDefaultKeyAlias() throws ExecutionException, InterruptedException
      {
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 -        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("UUIDKeys");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_UUID);
  
 -        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "UUIDKeys");
 +        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF_UUID);
          cf.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
 -        Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
 +        Mutation mutation = new Mutation(KEYSPACE, ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
          mutation.applyUnsafe();
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          assertEquals(1, cfs.getSSTables().size());
      }
@@@ -634,19 -545,19 +634,19 @@@
      @Test
      public void testValidationCompactStorage() throws Exception
      {
 -        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", ConsistencyLevel.ONE);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", KEYSPACE), ConsistencyLevel.ONE);
  
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_dynamic_columns");
  
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')");
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
 -        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')", KEYSPACE));
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')", KEYSPACE));
 +        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')", KEYSPACE));
          cfs.forceBlockingFlush();
-         CompactionManager.instance.performScrub(cfs, true, true);
+         CompactionManager.instance.performScrub(cfs, true, true, 2);
  
          // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
 -        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".test_compact_dynamic_columns", KEYSPACE));
          assertEquals(3, rs.size());
  
          Iterator<UntypedResultSet.Row> iter = rs.iterator();
@@@ -654,129 -565,4 +654,129 @@@
          assertEquals("bar", iter.next().getString("c"));
          assertEquals("boo", iter.next().getString("c"));
      }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
 +        // otherwise it uses LocalByPartitionerType
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
 +    }
 +
 +    /** The SecondaryIndex class is used for custom indexes so to avoid
 +     * making a public final field into a private field with getters
 +     * and setters, we resort to this hack in order to test it properly
 +     * since it can have two values which influence the scrubbing behavior.
 +     * @param comparator - the key comparator we want to test
 +     */
 +    private void setKeyComparator(AbstractType<?> comparator)
 +    {
 +        try
 +        {
 +            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
 +            keyComparator.setAccessible(true);
 +            int modifiers = keyComparator.getModifiers();
 +            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
 +            modifiers = modifiers & ~Modifier.FINAL;
 +            modifierField.setAccessible(true);
 +            modifierField.setInt(keyComparator, modifiers);
 +
 +            keyComparator.set(null, comparator);
 +        }
 +        catch (Exception ex)
 +        {
 +            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
 +            ex.printStackTrace();
 +        }
 +    }
 +
 +    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
 +            throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.instance.disableAutoCompaction();
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 +        cfs.clearUnsafe();
 +
 +        int numRows = 1000;
 +        long[] colValues = new long [numRows * 2]; // each row has two columns
 +        for (int i = 0; i < colValues.length; i+=2)
 +        {
 +            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
 +            colValues[i+1] = 3L; //other column
 +        }
 +        fillIndexCF(cfs, composite, colValues);
 +
 +        // check index
 +        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
 +        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +
 +        // scrub index
 +        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
 +        assertTrue(indexCfss.size() == 1);
 +        for(ColumnFamilyStore indexCfs : indexCfss)
 +        {
 +            for (int i = 0; i < scrubs.length; i++)
 +            {
 +                boolean failure = !scrubs[i];
 +                if (failure)
 +                { //make sure the next scrub fails
 +                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
 +                assertEquals(failure ?
 +                             CompactionManager.AllSSTableOpStatus.ABORTED :
 +                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
 +                                result);
 +            }
 +        }
 +
 +
 +        // check index is still working
 +        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/897ffe87/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------


[10/15] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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

Branch: refs/heads/trunk
Commit: 0c91977da68f054cf0ddf996b49423d736b42006
Parents: 3efc609 897ffe8
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Mar 29 11:00:13 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 11:00:13 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 16 ++++----
 .../db/compaction/CompactionManager.java        | 40 +++++++++++++-------
 .../cassandra/service/StorageService.java       | 23 +++++++++--
 .../cassandra/service/StorageServiceMBean.java  |  6 +++
 .../org/apache/cassandra/tools/NodeProbe.java   | 34 +++++++++++------
 .../cassandra/tools/nodetool/Cleanup.java       |  9 ++++-
 .../apache/cassandra/tools/nodetool/Scrub.java  | 13 +++++--
 .../tools/nodetool/UpgradeSSTable.java          | 10 ++++-
 .../org/apache/cassandra/db/CleanupTest.java    |  6 +--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 16 ++++----
 11 files changed, 120 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index fe19df7,098d062..fdc873e
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -20,36 -8,13 +20,38 @@@ Merged from 2.2
   * Add missing newline at end of bin/cqlsh (CASSANDRA-11325)
   * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
   * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 - * Fix filtering on non-primary key columns for thrift static column families
 -   (CASSANDRA-6377)
   * Only log yaml config once, at startup (CASSANDRA-11217)
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 +Merged from 2.1:
++ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
++   many threads to use (CASSANDRA-11179)
 + * Backport CASSANDRA-10679 (CASSANDRA-9598)
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
 + * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 +
 +3.0.4
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * MV should only query complex columns included in the view (CASSANDRA-11069)
 + * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
 + * Add sstabledump tool (CASSANDRA-7464)
 + * Introduce backpressure for hints (CASSANDRA-10972)
 + * Fix ClusteringPrefix not being able to read tombstone range boundaries (CASSANDRA-11158)
 + * Prevent logging in sandboxed state (CASSANDRA-11033)
 + * Disallow drop/alter operations of UDTs used by UDAs (CASSANDRA-10721)
 + * Add query time validation method on Index (CASSANDRA-11043)
 + * Avoid potential AssertionError in mixed version cluster (CASSANDRA-11128)
 + * Properly handle hinted handoff after topology changes (CASSANDRA-5902)
 + * AssertionError when listing sstable files on inconsistent disk state (CASSANDRA-11156)
 + * Fix wrong rack counting and invalid conditions check for TokenAllocation
 +   (CASSANDRA-11139)
 + * Avoid creating empty hint files (CASSANDRA-11090)
 + * Fix leak detection strong reference loop using weak reference (CASSANDRA-11120)
 + * Configurie BatchlogManager to stop delayed tasks on shutdown (CASSANDRA-11062)
 + * Hadoop integration is incompatible with Cassandra Driver 3.0.0 (CASSANDRA-11001)
 + * Add dropped_columns to the list of schema table so it gets handled
 +   properly (CASSANDRA-11050)
 + * Fix NPE when using forceRepairRangeAsync without DC (CASSANDRA-11239)
 +Merged from 2.2:
   * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
   * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
   * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 7c46fcb,ca02747..f339660
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -434,10 -441,10 +448,10 @@@ public class CompactionManager implemen
              @Override
              public void execute(LifecycleTransaction txn) throws IOException
              {
 -                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
 +                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges, FBUtilities.nowInSeconds());
                  doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
              }
-         }, OperationType.CLEANUP);
+         }, jobs, OperationType.CLEANUP);
      }
  
      public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 3a62063,bca5996..75573ac
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2633,32 -2553,42 +2633,42 @@@ public class StorageService extends Not
          return Gossiper.instance.getCurrentGenerationNumber(FBUtilities.getBroadcastAddress());
      }
  
 -    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
++        return forceKeyspaceCleanup(0, keyspaceName, tables);
+     }
+ 
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
 -        if (keyspaceName.equals(SystemKeyspace.NAME))
 +        if (Schema.isSystemKeyspace(keyspaceName))
              throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
  
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup(jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
          return status.statusCode;
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, tables);
 -        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, tables);
+     }
+ 
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }
@@@ -2677,12 -2606,17 +2687,17 @@@
          return status.statusCode;
      }
  
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
++        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 0, tableNames);
+     }
+ 
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, tableNames))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 049b4f9,761eed6..5b7331d
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -251,7 -251,9 +251,9 @@@ public interface StorageServiceMBean ex
      /**
       * Trigger a cleanup of keys on a single keyspace
       */
+     @Deprecated
 -    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException;
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@@ -260,8 -262,10 +262,10 @@@
       * Scrubbed CFs will be snapshotted first, if disableSnapshot is false
       */
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Verify (checksums of) the given keyspace.
@@@ -275,7 -279,9 +279,9 @@@
       * Rewrite all sstables to the latest version.
       * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
       */
+     @Deprecated
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException;
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Flush all memtables for the given column families, or all columnfamilies for the given keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index cdce497,2f27cea..a9d71d8
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -53,9 -53,9 +53,10 @@@ import javax.management.remote.JMXConne
  import javax.management.remote.JMXServiceURL;
  import javax.rmi.ssl.SslRMIClientSocketFactory;
  
 +import org.apache.cassandra.batchlog.BatchlogManager;
 +import org.apache.cassandra.batchlog.BatchlogManagerMBean;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 -import org.apache.cassandra.db.HintedHandOffManager;
  import org.apache.cassandra.db.HintedHandOffManagerMBean;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.compaction.CompactionManagerMBean;
@@@ -236,38 -231,46 +237,46 @@@ public class NodeProbe implements AutoC
          jmxc.close();
      }
  
-     public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.forceKeyspaceCleanup(keyspaceName, tables);
 -        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
++        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, tables);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, tables);
 -        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
++        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables);
      }
  
 -    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +        return ssProxy.verify(extendedVerify, keyspaceName, tableNames);
      }
  
-     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, tableNames);
 -        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
++        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames);
      }
  
-     public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     private void checkJobs(PrintStream out, int jobs)
      {
-         if (forceKeyspaceCleanup(keyspaceName, tableNames) != 0)
+         if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+             out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+     }
+ 
 -    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
+     {
+         checkJobs(out, jobs);
 -        if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
++        if (forceKeyspaceCleanup(jobs, keyspaceName, tableNames) != 0)
          {
              failed = true;
              out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, tables) != 0)
+         checkJobs(out, jobs);
 -        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
++        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables) != 0)
          {
              failed = true;
              out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
@@@ -284,9 -287,10 +293,10 @@@
      }
  
  
-     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
 -    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
-         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, tableNames) != 0)
+         checkJobs(out, jobs);
 -        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
++        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames) != 0)
          {
              failed = true;
              out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index cbfd4d6,6c6676d..a7507b1
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@@ -23,8 -23,8 +23,8 @@@ import io.airlift.command.Command
  import java.util.ArrayList;
  import java.util.List;
  
 +import org.apache.cassandra.config.Schema;
- import org.apache.cassandra.db.SystemKeyspace;
+ import io.airlift.command.Option;
 -import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.tools.NodeProbe;
  import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
  
@@@ -47,9 -52,8 +52,9 @@@ public class Cleanup extends NodeToolCm
  
              try
              {
-                 probe.forceKeyspaceCleanup(System.out, keyspace, tableNames);
 -                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
 -            } catch (Exception e)
++                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, tableNames);
 +            }
 +            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during cleanup", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index f92ae0d,dafe8d1..2345a85
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -58,11 -63,11 +63,13 @@@ public class Scrub extends NodeToolCm
          {
              try
              {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, tableNames);
 -                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
--            } catch (IllegalArgumentException e)
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, tableNames);
++            }
++            catch (IllegalArgumentException e)
              {
                  throw e;
--            } catch (Exception e)
++            }
++            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during scrubbing", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
index a775878,596f353..82866e0
--- a/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
@@@ -46,8 -51,8 +51,9 @@@ public class UpgradeSSTable extends Nod
          {
              try
              {
-                 probe.upgradeSSTables(System.out, keyspace, !includeAll, tableNames);
 -                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
--            } catch (Exception e)
++                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, tableNames);
++            }
++            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during enabling auto-compaction", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/CleanupTest.java
index 167f3b0,5777af4..b4ffe57
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@@ -92,11 -94,11 +92,11 @@@ public class CleanupTes
          // record max timestamps of the sstables pre-cleanup
          List<Long> expectedMaxTimestamps = getMaxTimestampList(cfs);
  
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(LOOPS, rows.size());
 +        iter = Util.getRangeSlice(cfs);
 +        assertEquals(LOOPS, Iterators.size(iter));
  
          // with one token in the ring, owned by the local node, cleanup should be a no-op
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
          // ensure max timestamp of the sstables are retained post-cleanup
          assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
@@@ -137,16 -143,18 +137,16 @@@
          tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
          tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
  
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
          // row data should be gone
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
  
          // not only should it be gone but there should be no data on disk, not even tombstones
 -        assert cfs.getSSTables().isEmpty();
 +        assert cfs.getLiveSSTables().isEmpty();
  
          // 2ary indexes should result in no results, too (although tombstones won't be gone until compacted)
 -        rows = cfs.search(range, clause, filter, Integer.MAX_VALUE);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).filterOn("birthdate", Operator.EQ, VALUE).build()).size());
      }
  
      @Test
@@@ -168,9 -180,10 +168,9 @@@
          tk2[0] = 1;
          tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
          tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
-         CompactionManager.instance.performCleanup(cfs);
+         CompactionManager.instance.performCleanup(cfs, 2);
  
 -        rows = Util.getRangeSlice(cfs);
 -        assertEquals(0, rows.size());
 +        assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
      }
  
      @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c91977d/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 6dbbb1b,4cca7ff..936ccd8
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -110,14 -127,18 +110,14 @@@ public class ScrubTes
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 1);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
      }
  
      @Test
@@@ -235,17 -260,21 +235,17 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 4);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(4, rows.size());
 +        assertOrderedAll(cfs, 4);
  
 -        SSTableReader sstable = cfs.getSSTables().iterator().next();
 +        SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
          overrideWithGarbage(sstable, 0, 2);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(4, rows.size());
 +        assertOrderedAll(cfs, 4);
      }
  
      @Test
@@@ -271,14 -318,18 +271,14 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 10);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
      }
  
      @Test
@@@ -289,17 -340,21 +289,17 @@@
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 10);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
  
 -        for (SSTableReader sstable : cfs.getSSTables())
 +        for (SSTableReader sstable : cfs.getLiveSSTables())
              new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)).delete();
  
-         CompactionManager.instance.performScrub(cfs, false, true);
+         CompactionManager.instance.performScrub(cfs, false, true, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(10, rows.size());
 +        assertOrderedAll(cfs, 10);
      }
  
      @Test
@@@ -504,11 -599,13 +504,11 @@@
  
          QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
          ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
 -        Mutation mutation = new Mutation("Keyspace1", UTF8Type.instance.decompose("key"));
 -        CellNameType ct = cfs2.getComparator();
 -        mutation.add("test_scrub_validation", ct.makeCellName("b"), LongType.instance.decompose(1L), System.currentTimeMillis());
 -        mutation.apply();
 +
 +        new Mutation(UpdateBuilder.create(cfs2.metadata, "key").newRow().add("b", LongType.instance.decompose(1L)).build()).apply();
          cfs2.forceBlockingFlush();
  
-         CompactionManager.instance.performScrub(cfs2, false, false);
+         CompactionManager.instance.performScrub(cfs2, false, false, 2);
      }
  
      /**
@@@ -614,9 -763,9 +614,9 @@@
                  boolean failure = !scrubs[i];
                  if (failure)
                  { //make sure the next scrub fails
 -                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                    overrideWithGarbage(indexCfs.getLiveSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
                  }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true);
+                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
                  assertEquals(failure ?
                               CompactionManager.AllSSTableOpStatus.ABORTED :
                               CompactionManager.AllSSTableOpStatus.SUCCESSFUL,


[03/15] cassandra git commit: Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Posted by ma...@apache.org.
Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Patch by marcuse; reviewed by Carl Yeksigian for CASSANDRA-11179


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

Branch: refs/heads/trunk
Commit: 8b8a3f5b99fa7a8eefed14cd7e41b81773617046
Parents: a9b5422
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Mar 14 11:01:11 2016 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:50:38 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 12 +++----
 .../db/compaction/CompactionManager.java        | 35 +++++++++++++++-----
 .../cassandra/service/StorageService.java       | 24 +++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 ++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 33 +++++++++++-------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++--
 .../org/apache/cassandra/db/CleanupTest.java    |  6 ++--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 18 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 10 files changed, 111 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5f01114..7794d4f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,6 +17,8 @@
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)
  * Avoid major compaction mixing repaired and unrepaired sstables in DTCS (CASSANDRA-11113)
  * test_bulk_round_trip_blogposts is failing occasionally (CASSANDRA-10938)
+ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
+   many threads to use (CASSANDRA-11179)
 
 
 2.1.13

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 a78f33f..3d66d3a 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1478,22 +1478,22 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return maxFile;
     }
 
-    public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
     }
 
     public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 ec7cb45..e382cab 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -271,7 +271,17 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation) throws ExecutionException, InterruptedException
+    /**
+     * Run an operation over all sstables using jobs threads
+     *
+     * @param cfs the column family store to run the operation on
+     * @param operation the operation to run
+     * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+     * @return status of the operation
+     * @throws ExecutionException
+     * @throws InterruptedException
+     */
+    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
     {
         Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
         if (compactingSSTables == null)
@@ -299,7 +309,8 @@ public class CompactionManager implements CompactionManagerMBean
                     logger.info("Executor has shut down, not submitting task");
                     return AllSSTableOpStatus.ABORTED;
                 }
-                futures.add(executor.submit(new Callable<Object>()
+
+                Callable<Object> callable = new Callable<Object>()
                 {
                     @Override
                     public Object call() throws Exception
@@ -315,7 +326,13 @@ public class CompactionManager implements CompactionManagerMBean
                         }
                         return this;
                     }
-                }));
+                };
+                futures.add(executor.submit(callable));
+                if (jobs > 0 && futures.size() == jobs)
+                {
+                    FBUtilities.waitOnFutures(futures);
+                    futures.clear();
+                }
             }
             FBUtilities.waitOnFutures(futures);
         }
@@ -341,7 +358,7 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
@@ -357,10 +374,10 @@ public class CompactionManager implements CompactionManagerMBean
             {
                 scrubOne(cfs, input, skipCorrupted, checkData);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
     {
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -385,10 +402,10 @@ public class CompactionManager implements CompactionManagerMBean
                 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                 task.execute(metrics);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfStore.isIndex();
         Keyspace keyspace = cfStore.keyspace;
@@ -416,7 +433,7 @@ public class CompactionManager implements CompactionManagerMBean
                 CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
                 doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
             }
-        });
+        }, jobs);
     }
 
     public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 98e2251..507aedb 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2385,13 +2385,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+    }
+
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         if (keyspaceName.equals(Keyspace.SYSTEM_KS))
             throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
 
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup(jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
@@ -2400,27 +2405,36 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, columnFamilies);
+        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
     }
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
         return status.statusCode;
     }
-
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+    }
+
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8fa2433..d3a1725 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -260,7 +260,9 @@ public interface StorageServiceMBean extends NotificationEmitter
     /**
      * Trigger a cleanup of keys on a single keyspace
      */
+    @Deprecated
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@ -270,13 +272,17 @@ public interface StorageServiceMBean extends NotificationEmitter
      */
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
      */
+    @Deprecated
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Flush all memtables for the given column families, or all columnfamilies for the given keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 1ad1147..ab08e9f 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -63,6 +63,7 @@ import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.HintedHandOffManager;
 import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@ -237,42 +238,50 @@ public class NodeProbe implements AutoCloseable
         jmxc.close();
     }
 
-    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
     }
 
-    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
     }
 
-    public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    private void checkJobs(PrintStream out, int jobs)
     {
-        if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+        if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+            out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+    }
+
+    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
+        if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 819049e..1de2e20 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -1138,6 +1138,11 @@ public class NodeTool
         @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
         private List<String> args = new ArrayList<>();
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to cleanup simultaneusly, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1151,7 +1156,7 @@ public class NodeTool
 
                 try
                 {
-                    probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
+                    probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during cleanup", e);
@@ -1267,6 +1272,11 @@ public class NodeTool
                 description = "Do not validate columns using column validator")
         private boolean noValidation = false;
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1277,7 +1287,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -1345,6 +1355,11 @@ public class NodeTool
         @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
         private boolean includeAll = false;
 
+        @Option(title = "jobs",
+                name = {"-j","--jobs"},
+                description = "Number of sstables to upgrade simultaneously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1355,7 +1370,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
+                    probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during enabling auto-compaction", e);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 1d04dfa..7f54ed7 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -83,7 +83,7 @@ public class CleanupTest extends SchemaLoader
         assertEquals(LOOPS, rows.size());
 
         // with one token in the ring, owned by the local node, cleanup should be a no-op
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // ensure max timestamp of the sstables are retained post-cleanup
         assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
@@ -128,7 +128,7 @@ public class CleanupTest extends SchemaLoader
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
 
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // row data should be gone
         rows = Util.getRangeSlice(cfs);
@@ -165,7 +165,7 @@ public class CleanupTest extends SchemaLoader
         tk2[0] = 1;
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         rows = Util.getRangeSlice(cfs);
         assertEquals(0, rows.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 167671b..4efd082 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -99,7 +99,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(1, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -228,7 +228,7 @@ public class ScrubTest extends SchemaLoader
         SSTableReader sstable = cfs.getSSTables().iterator().next();
         overrideWithGarbage(sstable, 0, 2);
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -249,7 +249,7 @@ public class ScrubTest extends SchemaLoader
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
         assert cfs.getSSTables().isEmpty();
     }
 
@@ -268,7 +268,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(10, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -293,7 +293,7 @@ public class ScrubTest extends SchemaLoader
         for (SSTableReader sstable : cfs.getSSTables())
             new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)).delete();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -506,7 +506,7 @@ public class ScrubTest extends SchemaLoader
 
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
         ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@ -516,7 +516,7 @@ public class ScrubTest extends SchemaLoader
         mutation.apply();
         cfs2.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs2, false, false);
+        CompactionManager.instance.performScrub(cfs2, false, false, 2);
     }
 
     /**
@@ -533,7 +533,7 @@ public class ScrubTest extends SchemaLoader
         Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
         mutation.applyUnsafe();
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         assertEquals(1, cfs.getSSTables().size());
     }
@@ -554,7 +554,7 @@ public class ScrubTest extends SchemaLoader
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, true, true);
+        CompactionManager.instance.performScrub(cfs, true, true, 2);
 
         // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
         UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 7d33c11..749056c 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -328,7 +328,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
         cfs.disableAutoCompaction();
-        cfs.sstablesRewrite(false);
+        cfs.sstablesRewrite(false, 2);
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
     }


[05/15] cassandra git commit: Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Posted by ma...@apache.org.
Add a -j parameter to scrub/cleanup/upgradesstables to state how many threads to use

Patch by marcuse; reviewed by Carl Yeksigian for CASSANDRA-11179


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

Branch: refs/heads/cassandra-3.5
Commit: 8b8a3f5b99fa7a8eefed14cd7e41b81773617046
Parents: a9b5422
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Mar 14 11:01:11 2016 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Mar 29 10:50:38 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 12 +++----
 .../db/compaction/CompactionManager.java        | 35 +++++++++++++++-----
 .../cassandra/service/StorageService.java       | 24 +++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  6 ++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 33 +++++++++++-------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++--
 .../org/apache/cassandra/db/CleanupTest.java    |  6 ++--
 .../unit/org/apache/cassandra/db/ScrubTest.java | 18 +++++-----
 .../LeveledCompactionStrategyTest.java          |  2 +-
 10 files changed, 111 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5f01114..7794d4f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,6 +17,8 @@
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)
  * Avoid major compaction mixing repaired and unrepaired sstables in DTCS (CASSANDRA-11113)
  * test_bulk_round_trip_blogposts is failing occasionally (CASSANDRA-10938)
+ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
+   many threads to use (CASSANDRA-11179)
 
 
 2.1.13

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 a78f33f..3d66d3a 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1478,22 +1478,22 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return maxFile;
     }
 
-    public CompactionManager.AllSSTableOpStatus forceCleanup() throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus forceCleanup(int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+        return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
     {
-        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion);
+        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
     }
 
     public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/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 ec7cb45..e382cab 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -271,7 +271,17 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation) throws ExecutionException, InterruptedException
+    /**
+     * Run an operation over all sstables using jobs threads
+     *
+     * @param cfs the column family store to run the operation on
+     * @param operation the operation to run
+     * @param jobs the number of threads to use - 0 means use all available. It never uses more than concurrent_compactors threads
+     * @return status of the operation
+     * @throws ExecutionException
+     * @throws InterruptedException
+     */
+    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
     {
         Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
         if (compactingSSTables == null)
@@ -299,7 +309,8 @@ public class CompactionManager implements CompactionManagerMBean
                     logger.info("Executor has shut down, not submitting task");
                     return AllSSTableOpStatus.ABORTED;
                 }
-                futures.add(executor.submit(new Callable<Object>()
+
+                Callable<Object> callable = new Callable<Object>()
                 {
                     @Override
                     public Object call() throws Exception
@@ -315,7 +326,13 @@ public class CompactionManager implements CompactionManagerMBean
                         }
                         return this;
                     }
-                }));
+                };
+                futures.add(executor.submit(callable));
+                if (jobs > 0 && futures.size() == jobs)
+                {
+                    FBUtilities.waitOnFutures(futures);
+                    futures.clear();
+                }
             }
             FBUtilities.waitOnFutures(futures);
         }
@@ -341,7 +358,7 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
@@ -357,10 +374,10 @@ public class CompactionManager implements CompactionManagerMBean
             {
                 scrubOne(cfs, input, skipCorrupted, checkData);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
     {
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -385,10 +402,10 @@ public class CompactionManager implements CompactionManagerMBean
                 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                 task.execute(metrics);
             }
-        });
+        }, jobs);
     }
 
-    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfStore.isIndex();
         Keyspace keyspace = cfStore.keyspace;
@@ -416,7 +433,7 @@ public class CompactionManager implements CompactionManagerMBean
                 CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
                 doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
             }
-        });
+        }, jobs);
     }
 
     public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 98e2251..507aedb 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2385,13 +2385,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return forceKeyspaceCleanup(0, keyspaceName, columnFamilies);
+    }
+
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         if (keyspaceName.equals(Keyspace.SYSTEM_KS))
             throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
 
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup(jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
@@ -2400,27 +2405,36 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, columnFamilies);
+        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
     }
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }
         return status.statusCode;
     }
-
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
+    }
+
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8fa2433..d3a1725 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -260,7 +260,9 @@ public interface StorageServiceMBean extends NotificationEmitter
     /**
      * Trigger a cleanup of keys on a single keyspace
      */
+    @Deprecated
     public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@ -270,13 +272,17 @@ public interface StorageServiceMBean extends NotificationEmitter
      */
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
      */
+    @Deprecated
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Flush all memtables for the given column families, or all columnfamilies for the given keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 1ad1147..ab08e9f 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -63,6 +63,7 @@ import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.HintedHandOffManager;
 import org.apache.cassandra.db.HintedHandOffManagerMBean;
@@ -237,42 +238,50 @@ public class NodeProbe implements AutoCloseable
         jmxc.close();
     }
 
-    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
     }
 
-    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
     }
 
-    public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    private void checkJobs(PrintStream out, int jobs)
     {
-        if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+        if (jobs > DatabaseDescriptor.getConcurrentCompactors())
+            out.println(String.format("jobs (%d) is bigger than configured concurrent_compactors (%d), using at most %d threads", jobs, DatabaseDescriptor.getConcurrentCompactors(), DatabaseDescriptor.getConcurrentCompactors()));
+    }
+
+    public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
+        if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+        checkJobs(out, jobs);
+        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 819049e..1de2e20 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -1138,6 +1138,11 @@ public class NodeTool
         @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
         private List<String> args = new ArrayList<>();
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to cleanup simultaneusly, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1151,7 +1156,7 @@ public class NodeTool
 
                 try
                 {
-                    probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
+                    probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during cleanup", e);
@@ -1267,6 +1272,11 @@ public class NodeTool
                 description = "Do not validate columns using column validator")
         private boolean noValidation = false;
 
+        @Option(title = "jobs",
+                name = {"-j", "--jobs"},
+                description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1277,7 +1287,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -1345,6 +1355,11 @@ public class NodeTool
         @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
         private boolean includeAll = false;
 
+        @Option(title = "jobs",
+                name = {"-j","--jobs"},
+                description = "Number of sstables to upgrade simultaneously, set to 0 to use all available compaction threads")
+        private int jobs = 2;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1355,7 +1370,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
+                    probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during enabling auto-compaction", e);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 1d04dfa..7f54ed7 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -83,7 +83,7 @@ public class CleanupTest extends SchemaLoader
         assertEquals(LOOPS, rows.size());
 
         // with one token in the ring, owned by the local node, cleanup should be a no-op
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // ensure max timestamp of the sstables are retained post-cleanup
         assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
@@ -128,7 +128,7 @@ public class CleanupTest extends SchemaLoader
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
 
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         // row data should be gone
         rows = Util.getRangeSlice(cfs);
@@ -165,7 +165,7 @@ public class CleanupTest extends SchemaLoader
         tk2[0] = 1;
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
-        CompactionManager.instance.performCleanup(cfs);
+        CompactionManager.instance.performCleanup(cfs, 2);
 
         rows = Util.getRangeSlice(cfs);
         assertEquals(0, rows.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 167671b..4efd082 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -99,7 +99,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(1, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -228,7 +228,7 @@ public class ScrubTest extends SchemaLoader
         SSTableReader sstable = cfs.getSSTables().iterator().next();
         overrideWithGarbage(sstable, 0, 2);
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -249,7 +249,7 @@ public class ScrubTest extends SchemaLoader
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
         assert cfs.getSSTables().isEmpty();
     }
 
@@ -268,7 +268,7 @@ public class ScrubTest extends SchemaLoader
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(10, rows.size());
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -293,7 +293,7 @@ public class ScrubTest extends SchemaLoader
         for (SSTableReader sstable : cfs.getSSTables())
             new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)).delete();
 
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         // check data is still there
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
@@ -506,7 +506,7 @@ public class ScrubTest extends SchemaLoader
 
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
         ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
@@ -516,7 +516,7 @@ public class ScrubTest extends SchemaLoader
         mutation.apply();
         cfs2.forceBlockingFlush();
 
-        CompactionManager.instance.performScrub(cfs2, false, false);
+        CompactionManager.instance.performScrub(cfs2, false, false, 2);
     }
 
     /**
@@ -533,7 +533,7 @@ public class ScrubTest extends SchemaLoader
         Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
         mutation.applyUnsafe();
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, false, true);
+        CompactionManager.instance.performScrub(cfs, false, true, 2);
 
         assertEquals(1, cfs.getSSTables().size());
     }
@@ -554,7 +554,7 @@ public class ScrubTest extends SchemaLoader
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
         QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
         cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, true, true);
+        CompactionManager.instance.performScrub(cfs, true, true, 2);
 
         // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
         UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b8a3f5b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 7d33c11..749056c 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -328,7 +328,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
         cfs.disableAutoCompaction();
-        cfs.sstablesRewrite(false);
+        cfs.sstablesRewrite(false, 2);
         assertTrue(strategy.getAllLevelSize()[1] > 0);
 
     }