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);
}