This is an automated email from the ASF dual-hosted git repository.
mck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/trunk by this push:
new 53fff0034f Further Improve debug around paused and disabled compaction
53fff0034f is described below
commit 53fff0034ffe8b5b7ac51dca40cbb2298757eab8
Author: Mick Semb Wever <[email protected]>
AuthorDate: Sat Dec 7 14:16:35 2024 +0400
Further Improve debug around paused and disabled compaction
patch by Mick Semb Wever; reviewed by Paulo Motta for CASSANDRA-19728
---
CHANGES.txt | 1 +
.../org/apache/cassandra/db/ColumnFamilyStore.java | 32 ++++++++++++++++++----
2 files changed, 27 insertions(+), 6 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index 1841ef66be..9e31f5ea19 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
5.1
+ * Improve debug around paused and disabled compaction
(CASSANDRA-20131,CASSANDRA-19728)
* DiskUsageBroadcaster does not update usageInfo on node replacement
(CASSANDRA-21033)
* Reject PrepareJoin if tokens are already assigned (CASSANDRA-21006)
* Don't update registration status if node state for decommissioned peer is
found with the same address (CASSANDRA-21005)
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 839a21d55d..4cc2615cd4 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -65,6 +65,7 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
+import org.apache.commons.lang3.exception.ExceptionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -2664,7 +2665,6 @@ public class ColumnFamilyStore implements
ColumnFamilyStoreMBean, Memtable.Owner
// and so we only run one major compaction at a time
synchronized (this)
{
- logger.debug("Cancelling in-progress compactions for {}",
metadata.name);
Iterable<ColumnFamilyStore> toInterruptFor = interruptIndexes
? concatWithIndexes()
:
Collections.singleton(this);
@@ -2675,6 +2675,9 @@ public class ColumnFamilyStore implements
ColumnFamilyStoreMBean, Memtable.Owner
Iterable<TableMetadata> toInterruptForMetadata =
Iterables.transform(toInterruptFor, ColumnFamilyStore::metadata);
+ logger.debug("Cancelling in-progress compactions for {}.{} ({})
{}", metadata.keyspace, metadata.name,
+ debugToInterruptFor(toInterruptFor),
onelinerStackTrace(new Throwable()));
+
try (CompactionManager.CompactionPauser pause =
CompactionManager.instance.pauseGlobalCompaction();
CompactionManager.CompactionPauser pausedStrategies =
pauseCompactionStrategies(toInterruptFor))
{
@@ -2700,13 +2703,11 @@ public class ColumnFamilyStore implements
ColumnFamilyStoreMBean, Memtable.Owner
{
if
(cfs.getTracker().getCompacting().stream().anyMatch(sstablesPredicate))
{
- 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.name);
+ 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);
return null;
}
}
- logger.trace("Compactions successfully cancelled");
+ logger.debug("Compactions successfully cancelled for {}.{}",
metadata.keyspace, metadata.name);
// run our task
try
@@ -2761,7 +2762,26 @@ public class ColumnFamilyStore implements
ColumnFamilyStoreMBean, Memtable.Owner
return accumulate;
}
- public <T> T withAllSSTables(final OperationType operationType,
Function<LifecycleTransaction, T> op)
+ private static String debugToInterruptFor(Iterable<ColumnFamilyStore>
toInterruptFor)
+ {
+ StringBuilder debug = new StringBuilder();
+ for (ColumnFamilyStore cfs : toInterruptFor)
+
debug.append(cfs.getKeyspaceName()).append('.').append(cfs.getTableName()).append(',');
+ debug.setLength(debug.length() - 1);
+ return debug.toString();
+ }
+
+ private static String onelinerStackTrace(Throwable t)
+ {
+ return ExceptionUtils.getStackTrace(t)
+ .replace("java.lang.Throwable", "")
+ .replaceAll("at org[.]apache[.]cassandra[.]", "at ..")
+ .replaceAll("at [a-z].+\n", "")
+ .replaceAll("\n|\t", " ")
+ .replaceAll(" +", " ");
+ }
+
+public <T> T withAllSSTables(final OperationType operationType,
Function<LifecycleTransaction, T> op)
{
Callable<LifecycleTransaction> callable = () -> {
assert data.getCompacting().isEmpty() : data.getCompacting();
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]