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

dcapwell pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 82d0c3e99061ded088ceb6a7aaadd1e810b7c444
Merge: fd6f9e6 0076217
Author: David Capwell <[email protected]>
AuthorDate: Wed Aug 19 15:25:47 2020 -0700

    Merge branch 'cassandra-3.11' into trunk

 CHANGES.txt                                        |  1 +
 .../org/apache/cassandra/db/lifecycle/LogFile.java | 17 ++++++-
 .../cassandra/db/lifecycle/LogTransactionTest.java | 53 ++++++++++++++++++++++
 3 files changed, 69 insertions(+), 2 deletions(-)

diff --cc CHANGES.txt
index 3d9a9eb,eef996d..3e23053
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -20,7 -5,11 +20,8 @@@ Merged from 3.0
   * Remove broken 'defrag-on-read' optimization (CASSANDRA-15432)
   * Check for endpoint collision with hibernating nodes (CASSANDRA-14599)
   * Operational improvements and hardening for replica filtering protection 
(CASSANDRA-15907)
 - * stop_paranoid disk failure policy is ignored on CorruptSSTableException 
after node is up (CASSANDRA-15191)
 - * Forbid altering UDTs used in partition keys (CASSANDRA-15933)
   * Fix empty/null json string representation (CASSANDRA-15896)
 - * 3.x fails to start if commit log has range tombstones from a column which 
is also deleted (CASSANDRA-15970)
+  * Handle difference in timestamp precision between java8 and java11 in 
LogFIle.java (CASSANDRA-16050)
  Merged from 2.2:
   * Fix CQL parsing of collections when the column type is reversed 
(CASSANDRA-15814)
  
diff --cc test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
index e5ff138,c727241..2703e44
--- a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@@ -1106,6 -1105,59 +1106,59 @@@ public class LogTransactionTest extend
      }
  
      @Test
+     public void testTruncateFileUpdateTime() throws IOException
+     {
+         // Idea is that we truncate the actual modification time on disk 
after creating the log file.
+         // On java11 this would fail since we would have millisecond 
resolution in the log file, but
+         // then the file gives second resolution.
+         testTruncatedModificationTimesHelper(sstable ->
+                                   {
+                                       // increase the modification time of 
the Data file
+                                       for (String filePath : 
sstable.getAllFilePaths())
+                                       {
+                                           File f = new File(filePath);
+                                           long lastModified = 
f.lastModified();
+                                           f.setLastModified(lastModified - 
(lastModified % 1000));
+                                       }
+                                   });
+     }
+ 
+     private static void 
testTruncatedModificationTimesHelper(Consumer<SSTableReader> modifier) throws 
IOException
+     {
+         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
 -        File dataFolder = new 
Directories(cfs.metadata).getDirectoryForNewSSTables();
++        File dataFolder = new 
Directories(cfs.metadata()).getDirectoryForNewSSTables();
+         SSTableReader sstableOld = sstable(dataFolder, cfs, 0, 128);
+         SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);
+ 
+         // simulate tracking sstables with a committed transaction except the 
checksum will be wrong
+         LogTransaction log = new LogTransaction(OperationType.COMPACTION);
+         assertNotNull(log);
+ 
+         log.trackNew(sstableNew);
+         LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld);
+ 
+         //modify the old sstable files
+         modifier.accept(sstableOld);
+ 
+         //Fake a commit
+         log.txnFile().commit();
+ 
 -        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
++        LogTransaction.removeUnfinishedLeftovers(cfs.metadata());
+ 
+         // only the new files should be there
+         assertFiles(dataFolder.getPath(), 
Sets.newHashSet(sstableNew.getAllFilePaths()));
+         sstableNew.selfRef().release();
+ 
+         // complete the transaction to avoid LEAK errors
+         assertNull(log.complete(null));
+ 
+         assertFiles(dataFolder.getPath(), 
Sets.newHashSet(sstableNew.getAllFilePaths()));
+ 
+         // make sure to run the tidier to avoid any leaks in the logs
+         tidier.run();
+     }
+ 
+     @Test
      public void testGetTemporaryFilesSafeAfterObsoletion() throws Throwable
      {
          ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to