Skip to content

Commit 53fff00

Browse files
Further Improve debug around paused and disabled compaction
patch by Mick Semb Wever; reviewed by Paulo Motta for CASSANDRA-19728
1 parent da31410 commit 53fff00

File tree

2 files changed

+27
-6
lines changed

2 files changed

+27
-6
lines changed

CHANGES.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,5 @@
11
5.1
2+
* Improve debug around paused and disabled compaction (CASSANDRA-20131,CASSANDRA-19728)
23
* DiskUsageBroadcaster does not update usageInfo on node replacement (CASSANDRA-21033)
34
* Reject PrepareJoin if tokens are already assigned (CASSANDRA-21006)
45
* Don't update registration status if node state for decommissioned peer is found with the same address (CASSANDRA-21005)

src/java/org/apache/cassandra/db/ColumnFamilyStore.java

Lines changed: 26 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -65,6 +65,7 @@
6565
import com.google.common.collect.Iterables;
6666
import com.google.common.collect.Lists;
6767
import com.google.common.collect.Sets;
68+
import org.apache.commons.lang3.exception.ExceptionUtils;
6869
import org.slf4j.Logger;
6970
import org.slf4j.LoggerFactory;
7071

@@ -2664,7 +2665,6 @@ public <V> V runWithCompactionsDisabled(Callable<V> callable, Predicate<SSTableR
26642665
// and so we only run one major compaction at a time
26652666
synchronized (this)
26662667
{
2667-
logger.debug("Cancelling in-progress compactions for {}", metadata.name);
26682668
Iterable<ColumnFamilyStore> toInterruptFor = interruptIndexes
26692669
? concatWithIndexes()
26702670
: Collections.singleton(this);
@@ -2675,6 +2675,9 @@ public <V> V runWithCompactionsDisabled(Callable<V> callable, Predicate<SSTableR
26752675

26762676
Iterable<TableMetadata> toInterruptForMetadata = Iterables.transform(toInterruptFor, ColumnFamilyStore::metadata);
26772677

2678+
logger.debug("Cancelling in-progress compactions for {}.{} ({}) {}", metadata.keyspace, metadata.name,
2679+
debugToInterruptFor(toInterruptFor), onelinerStackTrace(new Throwable()));
2680+
26782681
try (CompactionManager.CompactionPauser pause = CompactionManager.instance.pauseGlobalCompaction();
26792682
CompactionManager.CompactionPauser pausedStrategies = pauseCompactionStrategies(toInterruptFor))
26802683
{
@@ -2700,13 +2703,11 @@ public <V> V runWithCompactionsDisabled(Callable<V> callable, Predicate<SSTableR
27002703
{
27012704
if (cfs.getTracker().getCompacting().stream().anyMatch(sstablesPredicate))
27022705
{
2703-
logger.warn("Unable to cancel in-progress compactions for {}. " +
2704-
"Perhaps there is an unusually large row in progress somewhere, or the system is simply overloaded.",
2705-
metadata.name);
2706+
logger.warn("Unable to cancel in-progress compactions for {}.{}. Perhaps there is an unusually large row in progress somewhere, or the system is simply overloaded.", metadata.keyspace, metadata.name);
27062707
return null;
27072708
}
27082709
}
2709-
logger.trace("Compactions successfully cancelled");
2710+
logger.debug("Compactions successfully cancelled for {}.{}", metadata.keyspace, metadata.name);
27102711

27112712
// run our task
27122713
try
@@ -2761,7 +2762,26 @@ private static Throwable resumeAll(Throwable accumulate, Iterable<ColumnFamilySt
27612762
return accumulate;
27622763
}
27632764

2764-
public <T> T withAllSSTables(final OperationType operationType, Function<LifecycleTransaction, T> op)
2765+
private static String debugToInterruptFor(Iterable<ColumnFamilyStore> toInterruptFor)
2766+
{
2767+
StringBuilder debug = new StringBuilder();
2768+
for (ColumnFamilyStore cfs : toInterruptFor)
2769+
debug.append(cfs.getKeyspaceName()).append('.').append(cfs.getTableName()).append(',');
2770+
debug.setLength(debug.length() - 1);
2771+
return debug.toString();
2772+
}
2773+
2774+
private static String onelinerStackTrace(Throwable t)
2775+
{
2776+
return ExceptionUtils.getStackTrace(t)
2777+
.replace("java.lang.Throwable", "")
2778+
.replaceAll("at org[.]apache[.]cassandra[.]", "at ..")
2779+
.replaceAll("at [a-z].+\n", "")
2780+
.replaceAll("\n|\t", " ")
2781+
.replaceAll(" +", " ");
2782+
}
2783+
2784+
public <T> T withAllSSTables(final OperationType operationType, Function<LifecycleTransaction, T> op)
27652785
{
27662786
Callable<LifecycleTransaction> callable = () -> {
27672787
assert data.getCompacting().isEmpty() : data.getCompacting();

0 commit comments

Comments
 (0)