Add additional logging around FileUtils and compaction leftover cleanup

Patch by David Capwell; Reviewed by Jon Haddad for CASSANDRA-15705
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 @@
             {
                 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 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 @@
     {
         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 @@
         {
             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 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 @@
     {
         try
         {
+            if (!StorageService.instance.isDaemonSetupCompleted())
+                logger.info("Deleting file during startup: {}", file);
+
             Files.delete(file.toPath());
         }
         catch (Throwable t)
@@ -442,6 +444,9 @@
 
     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 @@
     {
         for ( File file : files )
         {
+            if (!StorageService.instance.isDaemonSetupCompleted())
+                logger.info("Deleting file during startup: {}", file);
+
             file.delete();
         }
     }