This is an automated email from the ASF dual-hosted git repository.

marcuse pushed a commit to branch cassandra-4.1
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit d4fb51347ca44386a0307bbfe1860d7ef16859e5
Merge: b4484968bd f2e3a0a4fc
Author: Marcus Eriksson <marc...@apache.org>
AuthorDate: Mon May 12 13:22:01 2025 +0200

    Merge branch 'cassandra-4.0' into cassandra-4.1

 CHANGES.txt                                        |   1 +
 .../org/apache/cassandra/db/lifecycle/LogFile.java |  19 ++-
 .../apache/cassandra/db/lifecycle/LogRecord.java   |  70 ++++++++-
 .../cassandra/db/lifecycle/LogTransaction.java     |   2 +
 .../cassandra/db/lifecycle/LogTransactionTest.java | 175 ++++++++++++++++++++-
 5 files changed, 256 insertions(+), 11 deletions(-)

diff --cc CHANGES.txt
index 032a805144,65e3cb4de0..491ed9b958
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,13 -1,8 +1,14 @@@
 -4.0.18
 +4.1.9
 + * Grant permission on keyspaces system_views and system_virtual_schema not 
possible (CASSANDRA-20171)
 + * Fix mixed mode paxos ttl commit hang (CASSANDRA-20514)
 + * Fix paxos mixed mode infinite loop (CASSANDRA-20493)
 + * Optionally skip exception logging on invalid legacy protocol magic 
exception (CASSANDRA-19483)
 + * Fix SimpleClient ability to release acquired capacity (CASSANDRA-20202)
 + * Fix WaitQueue.Signal.awaitUninterruptibly may block forever if invoking 
thread is interrupted (CASSANDRA-20084)
 +Merged from 4.0:
+  * Handle sstable metadata stats file getting a new mtime after compaction 
has finished (CASSANDRA-18119)
   * Honor MAX_PARALLEL_TRANSFERS correctly (CASSANDRA-20532)
   * Updating a column with a new TTL but same expiration time is 
non-deterministic and causes repair mismatches. (CASSANDRA-20561)
 - * Grant permission on keyspaces system_views and system_virtual_schema not 
possible (CASSANDRA-20171)
   * Avoid computing prepared statement size for unprepared batches 
(CASSANDRA-20556)
   * Fix Dropwizard Meter causes timeouts when infrequently used 
(CASSANDRA-19332)
   * Update OWASP dependency checker to version 12.1.0 (CASSANDRA-20501)
diff --cc src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
index 45653c47a3,65ff470ac8..9d9b05d8cb
--- a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
@@@ -20,23 -20,29 +20,29 @@@
   */
  package org.apache.cassandra.db.lifecycle;
  
- 
 -import java.io.File;
 -import java.io.FilenameFilter;
  import java.nio.file.Path;
 -import java.nio.file.Paths;
  import java.util.*;
 +import java.util.function.BiPredicate;
  import java.util.regex.Matcher;
  import java.util.regex.Pattern;
  import java.util.stream.Collectors;
  import java.util.zip.CRC32;
  
+ import com.google.common.annotations.VisibleForTesting;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
  import org.apache.cassandra.io.sstable.Component;
+ import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.SSTable;
  import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.util.File;
  import org.apache.cassandra.io.util.FileUtils;
 +import org.apache.cassandra.io.util.PathUtils;
  import org.apache.cassandra.utils.FBUtilities;
  
+ import static org.apache.cassandra.io.sstable.Descriptor.TMP_EXT;
+ 
  /**
   * A decoded line in a transaction log file replica.
   *
@@@ -186,8 -202,48 +202,48 @@@ final class LogRecor
          return make(type, existingFiles, 0, absolutePath.get());
      }
  
+     /**
+      * We create a LogRecord based on the files on disk; there's some 
subtlety around how we handle stats files as the
+      * timestamp can be mutated by the async completion of compaction if 
things race with node shutdown. To work around this,
+      * we don't take the stats file timestamp into account when calculating 
nor using the timestamps for all the components
+      * as we build the LogRecord.
+      */
      public static LogRecord make(Type type, List<File> files, int minFiles, 
String absolutePath)
      {
+         return make(type, files, minFiles, absolutePath, 
INCLUDE_STATS_FOR_TESTS);
+     }
+ 
+     /**
+      * In most cases we skip including the stats file timestamp entirely as 
it can be mutated during anticompaction
+      * and thus "invalidate" the LogRecord. There is an edge case where we 
have a LogRecord that was written w/the wrong
+      * timestamp (i.e. included a mutated stats file) and we need the node to 
come up, so we need to expose the selective
+      * ability to either include the stats file timestamp or not.
+      *
+      * See {@link LogFile#verifyRecord}
+      */
+     static LogRecord make(Type type, List<File> files, int minFiles, String 
absolutePath, boolean includeStatsFile)
+     {
+         List<File> toVerify;
+         File statsFile = null;
+         if (!includeStatsFile && !files.isEmpty())
+         {
+             toVerify = new ArrayList<>(files.size() - 1);
+             for (File f : files)
+             {
 -                if (!f.getName().endsWith(TMP_EXT))
++                if (!f.name().endsWith(TMP_EXT))
+                 {
+                     Component component = 
Descriptor.fromFilenameWithComponent(f).right;
+                     if (component == Component.STATS)
+                         statsFile = f;
+                     else
+                         toVerify.add(f);
+                 }
+             }
+         }
+         else
+         {
+             toVerify = files;
+         }
          // CASSANDRA-11889: File.lastModified() returns a positive value only 
if the file exists, therefore
          // we filter by positive values to only consider the files that still 
exists right now, in case things
          // changed on disk since getExistingFiles() was called
diff --cc test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
index 88164555cb,15578d14d5..0ff2a1fda4
--- a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@@ -17,8 -17,9 +17,7 @@@
   */
  package org.apache.cassandra.db.lifecycle;
  
- 
 -import java.io.File;
  import java.io.IOException;
 -import java.io.RandomAccessFile;
  import java.io.UncheckedIOException;
  import java.nio.file.Files;
  import java.nio.file.NoSuchFileException;
@@@ -1266,10 -1264,172 +1267,172 @@@ public class LogTransactionTest extend
          logs.finish();
      }
  
+     @Test
+     public void testStatsTSMatchOnStart() throws Throwable
+     {
+         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+         File dataFolder = new 
Directories(cfs.metadata()).getDirectoryForNewSSTables();
+         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
+ 
+         try(LogTransaction log = new LogTransaction(OperationType.COMPACTION))
+         {
+             assertNotNull(log);
+             log.trackNew(sstable);
+ 
+             // Confirm we can remove leftovers when they match
+             LogTransaction.removeUnfinishedLeftovers(cfs.metadata());
+         }
+ 
+         File sFile = new 
File(sstable.descriptor.filenameFor(Component.STATS));
+         assertFalse("Found STATS file but expected it to be cleaned up.", 
Files.exists(sFile.toPath()));
+         sstable.selfRef().release();
+     }
+ 
+     @Test
+     public void testStatsTSMatchDuringList() throws Throwable
+     {
+         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+         File dataFolder = new 
Directories(cfs.metadata()).getDirectoryForNewSSTables();
+         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
+ 
+         try(LogTransaction log = new LogTransaction(OperationType.COMPACTION))
+         {
+             assertNotNull(log);
+             log.trackNew(sstable);
+ 
+             // Confirm we can successfully classify files when they match - 
this triggers the LogAwareFileLister verify
+             listFiles(dataFolder, Directories.OnTxnErr.THROW, 
Directories.FileType.FINAL);
+         }
+         sstable.selfRef().release();
+     }
+ 
+     @Test
+     public void testStatsTSMismatchDuringStart() throws Throwable
+     {
+         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+         File dataFolder = new 
Directories(cfs.metadata()).getDirectoryForNewSSTables();
+         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
+ 
+         File sFile = new 
File(sstable.descriptor.filenameFor(Component.STATS));
+         assertTrue("STATS file not created successfully in test setup", 
Files.exists(sFile.toPath()));
+ 
+         // Confirm we can remove leftovers even if the STATS file doesn't 
match
+         try(LogTransaction log = new LogTransaction(OperationType.COMPACTION))
+         {
+             assertNotNull(log);
+ 
+             // Need to flag the transaction as having a REMOVE entry so it'll 
trigger the path to calculate stats on list
+             log.obsoleted(sstable);
+ 
+             // Need to sleep for long enough to bypass the millisecond 
truncation logic due to jdk8 and jdk11 change
+             Thread.sleep(2000);
 -            assertTrue("Failed to set mtime for STATS file to 
currentTimeMillis()", sFile.setLastModified(System.currentTimeMillis()));
++            assertTrue("Failed to set mtime for STATS file to 
currentTimeMillis()", sFile.trySetLastModified(System.currentTimeMillis()));
+ 
+             // Confirm we have an mtime mismatch
+             File dFile = new 
File(sstable.descriptor.filenameFor(Component.DATA));
+             assertNotEquals(sFile.lastModified(), dFile.lastModified());
+ 
+             // We need to add another LogRecord as we allow partial or 
incorrect entries as the last record...
+             log.trackNew(sstable(dataFolder, cfs, 2, 128));
+ 
+             assertTrue("STATS file gone before removeUnfinished...", 
Files.exists(sFile.toPath()));
+             // Confirm we can remove leftovers when the STATS file mismatches
+             log.prepareToCommit(); // commit so that obsolete sstable 
components will be removed.
+             log.commit();
+             ComponentContext.create(sstable.descriptor);
+             
assertTrue(LogTransaction.removeUnfinishedLeftovers(cfs.metadata()));
+         }
+ 
+         sstable.selfRef().release();
+     }
+ 
+     @Test
+     public void testWrongTimestampInTxnFile() throws IOException, 
InterruptedException
+     {
+         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+         File dataFolder = new 
Directories(cfs.metadata()).getDirectoryForNewSSTables();
+         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
+ 
+         File sFile = new 
File(sstable.descriptor.filenameFor(Component.STATS));
+         assertTrue("STATS file not created successfully in test setup", 
Files.exists(sFile.toPath()));
+ 
+         LogRecord.INCLUDE_STATS_FOR_TESTS = true;
+ 
+         Thread.sleep(2000);
 -        assertTrue("Failed to set mtime for STATS file to 
currentTimeMillis()", sFile.setLastModified(System.currentTimeMillis()));
++        assertTrue("Failed to set mtime for STATS file to 
currentTimeMillis()", sFile.trySetLastModified(System.currentTimeMillis()));
+ 
+         // Confirm we can remove leftovers even if the STATS file doesn't 
match
+         try(LogTransaction log = new LogTransaction(OperationType.COMPACTION))
+         {
+             assertNotNull(log);
+             // Need to flag the transaction as having a REMOVE entry so it'll 
trigger the path to calculate stats on list
+             log.obsoleted(sstable);
+             // Need to sleep for long enough to bypass the millisecond 
truncation logic due to jdk8 and jdk11 change
+             // Confirm we have an mtime mismatch
+             File dFile = new 
File(sstable.descriptor.filenameFor(Component.DATA));
+             assertNotEquals(sFile.lastModified(), dFile.lastModified());
+ 
+             // We need to add another LogRecord as we allow partial or 
incorrect entries as the last record...
+             log.trackNew(sstable(dataFolder, cfs, 2, 128));
+ 
+             assertTrue("STATS file gone before removeUnfinished...", 
Files.exists(sFile.toPath()));
+             // Confirm we can remove leftovers when the STATS file mismatches
+             LogRecord.INCLUDE_STATS_FOR_TESTS = false;
+             
assertTrue(LogTransaction.removeUnfinishedLeftovers(cfs.metadata()));
+         }
+ 
+         sstable.selfRef().release();
+     }
+ 
+     /**
+      * We do not consider the stats file's ts for any cases at this point
+      */
+     @Test
+     public void testStatsTSMismatchDuringList() throws Throwable
+     {
+         SSTableReader sstable = null;
+         try
+         {
+             ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+             File dataFolder = new 
Directories(cfs.metadata()).getDirectoryForNewSSTables();
+             sstable = sstable(dataFolder, cfs, 0, 128);
+ 
+             File sFile = new 
File(sstable.descriptor.filenameFor(Component.STATS));
+             assertTrue("STATS file not created successfully in test setup", 
Files.exists(sFile.toPath()));
+ 
+             try(LogTransaction log = new 
LogTransaction(OperationType.COMPACTION))
+             {
+                 assertNotNull(log);
+ 
+                 // Need to flag the transaction as having a REMOVE entry so 
it'll trigger the path to calculate stats on list
+                 log.obsoleted(sstable);
+ 
+                 // Need to sleep for long enough to bypass the millisecond 
truncation logic due to jdk8 and jdk11 change
+                 Thread.sleep(2000);
 -                assertTrue("Failed to set mtime for STATS file to 
currentTimeMillis()", sFile.setLastModified(System.currentTimeMillis()));
++                assertTrue("Failed to set mtime for STATS file to 
currentTimeMillis()", sFile.trySetLastModified(System.currentTimeMillis()));
+ 
+                 // Confirm we have an mtime mismatch
+                 File dFile = new 
File(sstable.descriptor.filenameFor(Component.DATA));
+                 assertNotEquals(sFile.lastModified(), dFile.lastModified());
+ 
+                 // We need to add another LogRecord as we allow partial or 
incorrect entries as the last record...
+                 log.trackNew(sstable(dataFolder, cfs, 2, 128));
+ 
+                 // Confirm we don't get a mismatch LogRecord error when the 
STATS file is different even on listFiles case
+                 listFiles(dataFolder, Directories.OnTxnErr.THROW, 
Directories.FileType.FINAL);
+             }
+         }
+         finally
+         {
+             if (sstable != null)
+                 sstable.selfRef().release();
+         }
+     }
+ 
      private static SSTableReader sstable(File dataFolder, ColumnFamilyStore 
cfs, int generation, int size) throws IOException
      {
 -        Descriptor descriptor = new Descriptor(dataFolder, 
cfs.keyspace.getName(), cfs.getTableName(), generation, SSTableFormat.Type.BIG);
 +        Descriptor descriptor = new Descriptor(dataFolder, 
cfs.keyspace.getName(), cfs.getTableName(), new 
SequenceBasedSSTableId(generation), SSTableFormat.Type.BIG);
-         Set<Component> components = ImmutableSet.of(Component.DATA, 
Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
+         Set<Component> components = ImmutableSet.of(Component.DATA, 
Component.PRIMARY_INDEX, Component.FILTER, Component.TOC, Component.STATS);
          for (Component component : components)
          {
              File file = new File(descriptor.filenameFor(component));


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to