You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ru...@apache.org on 2020/04/09 23:09:10 UTC
[cassandra] branch trunk updated: Add additional logging around
FileUtils and compaction leftover cleanup
This is an automated email from the ASF dual-hosted git repository.
rustyrazorblade 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 e485636 Add additional logging around FileUtils and compaction leftover cleanup
e485636 is described below
commit e485636ff1ea53221c3b22b1b0e88fad0a009f2e
Author: David Capwell <dc...@gmail.com>
AuthorDate: Tue Apr 7 16:43:40 2020 -0700
Add additional logging around FileUtils and compaction leftover cleanup
Patch by David Capwell; Reviewed by Jon Haddad for CASSANDRA-15705
---
CHANGES.txt | 1 +
src/java/org/apache/cassandra/db/ColumnFamilyStore.java | 3 +++
.../org/apache/cassandra/db/lifecycle/LogTransaction.java | 6 +++++-
src/java/org/apache/cassandra/io/util/FileUtils.java | 14 +++++++++++---
4 files changed, 20 insertions(+), 4 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index 2ad2aab..88ba6cb 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
4.0-alpha4
+ * Add additional logging around FileUtils and compaction leftover cleanup (CASSANDRA-15705)
* Mark system_views/system_virtual_schema as non-alterable keyspaces in cqlsh (CASSANDRA-15711)
* Fail incremental repair if an old version sstable is involved (CASSANDRA-15612)
* Fix overflows on StreamingTombstoneHistogramBuilder produced by large deletion times (CASSANDRA-14773)
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index ea04c86..e6bb877 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -635,7 +635,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
cleanedDirectories.add(directory);
for (File tmpFile : desc.getTemporaryFiles())
+ {
+ logger.info("Removing unfinished temporary file {}", tmpFile);
tmpFile.delete();
+ }
}
File dataFile = new File(desc.filenameFor(Component.DATA));
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
index 4adff86..4039322 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@ -30,6 +30,7 @@ import java.util.function.Predicate;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.Runnables;
+import org.apache.cassandra.service.StorageService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -214,7 +215,9 @@ class LogTransaction extends Transactional.AbstractTransactional implements Tran
{
try
{
- if (logger.isTraceEnabled())
+ if (!StorageService.instance.isDaemonSetupCompleted())
+ logger.info("Unfinished transaction log, deleting {} ", file);
+ else if (logger.isTraceEnabled())
logger.trace("Deleting {}", file);
Files.delete(file.toPath());
@@ -479,6 +482,7 @@ class LogTransaction extends Transactional.AbstractTransactional implements Tran
{
try(LogFile txn = LogFile.make(entry.getKey(), entry.getValue()))
{
+ logger.info("Verifying logfile transaction {}", txn);
if (txn.verify())
{
Throwable failure = txn.removeUnfinishedLeftovers(null);
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 549e544..2be6b5e 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -43,16 +43,15 @@ import java.util.concurrent.atomic.AtomicReference;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.utils.SyncUtil;
-
import org.apache.cassandra.concurrent.ScheduledExecutors;
import org.apache.cassandra.io.FSError;
import org.apache.cassandra.io.FSErrorHandler;
import org.apache.cassandra.io.FSReadError;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.memory.MemoryUtil;
+import org.apache.cassandra.utils.SyncUtil;
import static com.google.common.base.Throwables.propagate;
import static org.apache.cassandra.utils.Throwables.maybeFail;
@@ -192,6 +191,9 @@ public final class FileUtils
{
try
{
+ if (!StorageService.instance.isDaemonSetupCompleted())
+ logger.info("Deleting file during startup: {}", file);
+
Files.delete(file.toPath());
}
catch (Throwable t)
@@ -442,6 +444,9 @@ public final class FileUtils
public static boolean delete(String file)
{
+ if (!StorageService.instance.isDaemonSetupCompleted())
+ logger.info("Deleting file during startup: {}", file);
+
File f = new File(file);
return f.delete();
}
@@ -450,6 +455,9 @@ public final class FileUtils
{
for ( File file : files )
{
+ if (!StorageService.instance.isDaemonSetupCompleted())
+ logger.info("Deleting file during startup: {}", file);
+
file.delete();
}
}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org