You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/09/08 12:56:54 UTC

[1/7] cassandra git commit: Handle non-atomic directory streams safely (CASSANDRA-10109)

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 3818d30bd -> 351c7caca
  refs/heads/trunk fba356d1d -> 322e21ec4


http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
new file mode 100644
index 0000000..a655fd8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@ -0,0 +1,823 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.*;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.BufferedSegmentedFile;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+public class LogTransactionTest extends AbstractTransactionalTest
+{
+    private static final String KEYSPACE = "TransactionLogsTest";
+
+    @BeforeClass
+    public static void setUp()
+    {
+        MockSchema.cleanup();
+    }
+
+    protected AbstractTransactionalTest.TestableTransaction newTest() throws Exception
+    {
+        LogTransaction.waitForDeletions();
+        SSTableReader.resetTidying();
+        return new TxnTest();
+    }
+
+    private static final class TxnTest extends TestableTransaction
+    {
+        private final static class Transaction extends Transactional.AbstractTransactional implements Transactional
+        {
+            final ColumnFamilyStore cfs;
+            final LogTransaction txnLogs;
+            final SSTableReader sstableOld;
+            final SSTableReader sstableNew;
+            final LogTransaction.SSTableTidier tidier;
+
+            Transaction(ColumnFamilyStore cfs, LogTransaction txnLogs) throws IOException
+            {
+                this.cfs = cfs;
+                this.txnLogs = txnLogs;
+                this.sstableOld = sstable(cfs, 0, 128);
+                this.sstableNew = sstable(cfs, 1, 128);
+
+                assertNotNull(txnLogs);
+                assertNotNull(txnLogs.getId());
+                Assert.assertEquals(OperationType.COMPACTION, txnLogs.getType());
+
+                txnLogs.trackNew(sstableNew);
+                tidier = txnLogs.obsoleted(sstableOld);
+                assertNotNull(tidier);
+            }
+
+            protected Throwable doCommit(Throwable accumulate)
+            {
+                sstableOld.markObsolete(tidier);
+                sstableOld.selfRef().release();
+                LogTransaction.waitForDeletions();
+
+                Throwable ret = txnLogs.commit(accumulate);
+
+                sstableNew.selfRef().release();
+                return ret;
+            }
+
+            protected Throwable doAbort(Throwable accumulate)
+            {
+                tidier.abort();
+                LogTransaction.waitForDeletions();
+
+                Throwable ret = txnLogs.abort(accumulate);
+
+                sstableNew.selfRef().release();
+                sstableOld.selfRef().release();
+                return ret;
+            }
+
+            protected void doPrepare()
+            {
+                txnLogs.prepareToCommit();
+            }
+
+            void assertInProgress() throws Exception
+            {
+                assertFiles(txnLogs.getDataFolder(), Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths(),
+                                                                                      sstableOld.getAllFilePaths(),
+                                                                                      Collections.singleton(txnLogs.getLogFile().file.getPath()))));
+            }
+
+            void assertPrepared() throws Exception
+            {
+            }
+
+            void assertAborted() throws Exception
+            {
+                assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
+            }
+
+            void assertCommitted() throws Exception
+            {
+                assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+            }
+        }
+
+        final Transaction txn;
+
+        private TxnTest() throws IOException
+        {
+            this(MockSchema.newCFS(KEYSPACE));
+        }
+
+        private TxnTest(ColumnFamilyStore cfs) throws IOException
+        {
+            this(cfs, new LogTransaction(OperationType.COMPACTION, cfs.metadata));
+        }
+
+        private TxnTest(ColumnFamilyStore cfs, LogTransaction txnLogs) throws IOException
+        {
+            this(new Transaction(cfs, txnLogs));
+        }
+
+        private TxnTest(Transaction txn)
+        {
+            super(txn);
+            this.txn = txn;
+        }
+
+        protected void assertInProgress() throws Exception
+        {
+            txn.assertInProgress();
+        }
+
+        protected void assertPrepared() throws Exception
+        {
+            txn.assertPrepared();
+        }
+
+        protected void assertAborted() throws Exception
+        {
+            txn.assertAborted();
+        }
+
+        protected void assertCommitted() throws Exception
+        {
+            txn.assertCommitted();
+        }
+    }
+
+    @Test
+    public void testUntrack() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        // complete a transaction without keep the new files since they were untracked
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        log.untrackNew(sstableNew);
+
+        log.finish();
+
+        sstableNew.selfRef().release();
+        Thread.sleep(1);
+        LogTransaction.waitForDeletions();
+
+        assertFiles(log.getDataFolder(), Collections.<String>emptySet());
+    }
+
+    @Test
+    public void testCommitSameDesc() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld1 = sstable(cfs, 0, 128);
+        SSTableReader sstableOld2 = sstable(cfs, 0, 256);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+
+        sstableOld1.setReplaced();
+
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld2);
+        assertNotNull(tidier);
+
+        log.finish();
+
+        sstableOld2.markObsolete(tidier);
+
+        sstableOld1.selfRef().release();
+        sstableOld2.selfRef().release();
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+
+        sstableNew.selfRef().release();
+    }
+
+    @Test
+    public void testCommitOnlyNew() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstable);
+        log.finish();
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
+
+        sstable.selfRef().release();
+    }
+
+    @Test
+    public void testCommitOnlyOld() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstable);
+        assertNotNull(tidier);
+
+        log.finish();
+        sstable.markObsolete(tidier);
+        sstable.selfRef().release();
+
+        assertFiles(log.getDataFolder(), new HashSet<>());
+    }
+
+    @Test
+    public void testAbortOnlyNew() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstable);
+        log.abort();
+
+        sstable.selfRef().release();
+
+        assertFiles(log.getDataFolder(), new HashSet<>());
+    }
+
+    @Test
+    public void testAbortOnlyOld() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstable);
+        assertNotNull(tidier);
+
+        tidier.abort();
+        log.abort();
+
+        sstable.selfRef().release();
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
+    }
+
+    @Test
+    public void testRemoveUnfinishedLeftovers_abort() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld = sstable(cfs, 0, 128);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        // simulate tracking sstables with a failed transaction (new log file NOT deleted)
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld);
+
+        Set<File> tmpFiles = sstableNew.getAllFilePaths().stream().map(File::new).collect(Collectors.toSet());
+
+        sstableNew.selfRef().release();
+        sstableOld.selfRef().release();
+
+        Assert.assertEquals(tmpFiles, LogAwareFileLister.getTemporaryFiles(sstableNew.descriptor.directory));
+
+        // normally called at startup
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+        // sstableOld should be only table left
+        Directories directories = new Directories(cfs.metadata);
+        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
+        assertEquals(1, sstables.size());
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
+
+        tidier.run();
+
+        // complete the transaction before releasing files
+        log.close();
+    }
+
+    @Test
+    public void testRemoveUnfinishedLeftovers_commit() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld = sstable(cfs, 0, 128);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        // simulate tracking sstables with a committed transaction (new log file deleted)
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld);
+
+        //Fake a commit
+        log.getLogFile().commit();
+
+        Set<File> tmpFiles = sstableOld.getAllFilePaths().stream().map(File::new).collect(Collectors.toSet());
+
+        sstableNew.selfRef().release();
+        sstableOld.selfRef().release();
+
+        Assert.assertEquals(tmpFiles, LogAwareFileLister.getTemporaryFiles(sstableOld.descriptor.directory));
+
+        // normally called at startup
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+        // sstableNew should be only table left
+        Directories directories = new Directories(cfs.metadata);
+        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
+        assertEquals(1, sstables.size());
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+
+        tidier.run();
+
+        // complete the transaction to avoid LEAK errors
+        assertNull(log.complete(null));
+    }
+
+    @Test
+    public void testGetTemporaryFiles() throws IOException
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable1 = sstable(cfs, 0, 128);
+
+        File dataFolder = sstable1.descriptor.directory;
+
+        Set<File> tmpFiles = LogAwareFileLister.getTemporaryFiles(dataFolder);
+        assertNotNull(tmpFiles);
+        assertEquals(0, tmpFiles.size());
+
+        try(LogTransaction log = new LogTransaction(OperationType.WRITE, cfs.metadata))
+        {
+            Directories directories = new Directories(cfs.metadata);
+
+            File[] beforeSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
+
+            SSTableReader sstable2 = sstable(cfs, 1, 128);
+            log.trackNew(sstable2);
+
+            Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
+            assertEquals(2, sstables.size());
+
+            // this should contain sstable1, sstable2 and the transaction log file
+            File[] afterSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
+
+            int numNewFiles = afterSecondSSTable.length - beforeSecondSSTable.length;
+            assertEquals(numNewFiles - 1, sstable2.getAllFilePaths().size()); // new files except for transaction log file
+
+            tmpFiles = LogAwareFileLister.getTemporaryFiles(dataFolder);
+            assertNotNull(tmpFiles);
+            assertEquals(numNewFiles - 1, tmpFiles.size());
+
+            File ssTable2DataFile = new File(sstable2.descriptor.filenameFor(Component.DATA));
+            File ssTable2IndexFile = new File(sstable2.descriptor.filenameFor(Component.PRIMARY_INDEX));
+
+            assertTrue(tmpFiles.contains(ssTable2DataFile));
+            assertTrue(tmpFiles.contains(ssTable2IndexFile));
+
+            List<File> files = directories.sstableLister(Directories.OnTxnErr.THROW).listFiles();
+            List<File> filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles();
+            assertNotNull(files);
+            assertNotNull(filesNoTmp);
+
+            assertTrue(files.contains(ssTable2DataFile));
+            assertTrue(files.contains(ssTable2IndexFile));
+
+            assertFalse(filesNoTmp.contains(ssTable2DataFile));
+            assertFalse(filesNoTmp.contains(ssTable2IndexFile));
+
+            log.finish();
+
+            //Now it should be empty since the transaction has finished
+            tmpFiles = LogAwareFileLister.getTemporaryFiles(dataFolder);
+            assertNotNull(tmpFiles);
+            assertEquals(0, tmpFiles.size());
+
+            filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles();
+            assertNotNull(filesNoTmp);
+            assertTrue(filesNoTmp.contains(ssTable2DataFile));
+            assertTrue(filesNoTmp.contains(ssTable2IndexFile));
+
+            sstable1.selfRef().release();
+            sstable2.selfRef().release();
+        }
+    }
+
+    @Test
+    public void testWrongChecksumLastLine() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake a commit with invalid checksum
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+                          },
+                          true);
+    }
+
+    @Test
+    public void testWrongChecksumSecondFromLastLine() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake two lines with invalid checksum
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("add:[ma-3-big,%d,4][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+                          },
+                          false);
+    }
+
+    @Test
+    public void testWrongChecksumLastLineMissingFile() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake a commit with invalid checksum and also delete one of the old files
+                              for (String filePath : s.getAllFilePaths())
+                              {
+                                  if (filePath.endsWith("Data.db"))
+                                  {
+                                      assertTrue(FileUtils.delete(filePath));
+                                      t.getLogFile().sync();
+                                      break;
+                                  }
+                              }
+
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+                          },
+                          false);
+    }
+
+    @Test
+    public void testWrongChecksumLastLineWrongRecordFormat() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake a commit with invalid checksum and a wrong record format (extra spaces)
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d ,0 ,0 ][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+                          },
+                          true);
+    }
+
+    @Test
+    public void testMissingChecksumLastLine() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          {
+                              // Fake a commit without a checksum
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0]",
+                                                             System.currentTimeMillis()));
+                          },
+                          true);
+    }
+
+    @Test
+    public void testMissingChecksumSecondFromLastLine() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake two lines without a checksum
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("add:[ma-3-big,%d,4]",
+                                                             System.currentTimeMillis()));
+
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0]",
+                                                             System.currentTimeMillis()));
+                          },
+                          false);
+    }
+
+    private static void testCorruptRecord(BiConsumer<LogTransaction, SSTableReader> modifier, boolean isRecoverable) throws IOException
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld = sstable(cfs, 0, 128);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        File dataFolder = sstableOld.descriptor.directory;
+
+        // simulate tracking sstables with a committed transaction except the checksum will be wrong
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        log.obsoleted(sstableOld);
+
+        // Modify the transaction log or disk state for sstableOld
+        modifier.accept(log, sstableOld);
+
+        String txnFilePath = log.getLogFile().file.getPath();
+
+        assertNull(log.complete(null));
+
+        sstableOld.selfRef().release();
+        sstableNew.selfRef().release();
+
+        // The files on disk, for old files make sure to exclude the files that were deleted by the modifier
+        Set<String> newFiles = sstableNew.getAllFilePaths().stream().collect(Collectors.toSet());
+        Set<String> oldFiles = sstableOld.getAllFilePaths().stream().filter(p -> new File(p).exists()).collect(Collectors.toSet());
+
+        //This should filter as in progress since the last record is corrupt
+        assertFiles(newFiles, LogAwareFileLister.getTemporaryFiles(dataFolder));
+        assertFiles(oldFiles, LogAwareFileLister.getFinalFiles(dataFolder));
+
+        if (isRecoverable)
+        { // the corruption is recoverable but the commit record is unreadable so the transaction is still in progress
+
+            //This should remove new files
+            LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+            // make sure to exclude the old files that were deleted by the modifier
+            assertFiles(dataFolder.getPath(), oldFiles);
+        }
+        else
+        { // if an intermediate line was also modified, it should ignore the tx log file
+
+            //This should not remove any files
+            LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+            assertFiles(dataFolder.getPath(), Sets.newHashSet(Iterables.concat(newFiles,
+                                                                               oldFiles,
+                                                                               Collections.singleton(txnFilePath))));
+        }
+    }
+
+    @Test
+    public void testObsoletedDataFileUpdateTimeChanged() throws IOException
+    {
+        testObsoletedFilesChanged(sstable ->
+                                  {
+                                      // increase the modification time of the Data file
+                                      for (String filePath : sstable.getAllFilePaths())
+                                      {
+                                          if (filePath.endsWith("Data.db"))
+                                              assertTrue(new File(filePath).setLastModified(System.currentTimeMillis() + 60000)); //one minute later
+                                      }
+                                  });
+    }
+
+    private static void testObsoletedFilesChanged(Consumer<SSTableReader> modifier) throws IOException
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld = sstable(cfs, 0, 128);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        // simulate tracking sstables with a committed transaction except the checksum will be wrong
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        /*TransactionLog.SSTableTidier tidier =*/ log.obsoleted(sstableOld);
+
+        //modify the old sstable files
+        modifier.accept(sstableOld);
+
+        //Fake a commit
+        log.getLogFile().commit();
+
+        //This should not remove the old files
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+        assertFiles(log.getDataFolder(), Sets.newHashSet(Iterables.concat(
+                                                                                    sstableNew.getAllFilePaths(),
+                                                                                    sstableOld.getAllFilePaths(),
+                                                                                    Collections.singleton(log.getLogFile().file.getPath()))));
+
+        sstableOld.selfRef().release();
+        sstableNew.selfRef().release();
+
+        // complete the transaction to avoid LEAK errors
+        assertNull(log.complete(null));
+
+        assertFiles(log.getDataFolder(), Sets.newHashSet(Iterables.concat(
+                                                                                    sstableNew.getAllFilePaths(),
+                                                                                    sstableOld.getAllFilePaths(),
+                                                                                    Collections.singleton(log.getLogFile().file.getPath()))));
+    }
+
+    @Test
+    public void testGetTemporaryFilesSafeAfterObsoletion() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+        File dataFolder = sstable.descriptor.directory;
+
+        LogTransaction logs = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(logs);
+
+        LogTransaction.SSTableTidier tidier = logs.obsoleted(sstable);
+
+        logs.finish();
+
+        sstable.markObsolete(tidier);
+        sstable.selfRef().release();
+
+        // This should race with the asynchronous deletion of txn log files
+        // It doesn't matter what it returns but it should not throw because the txn
+        // was completed before deleting files (i.e. releasing sstables)
+        for (int i = 0; i < 200; i++)
+            LogAwareFileLister.getTemporaryFiles(dataFolder);
+    }
+
+    @Test
+    public void testGetTemporaryFilesThrowsIfCompletingAfterObsoletion() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+        File dataFolder = sstable.descriptor.directory;
+
+        LogTransaction logs = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(logs);
+
+        LogTransaction.SSTableTidier tidier = logs.obsoleted(sstable);
+
+        sstable.markObsolete(tidier);
+        sstable.selfRef().release();
+
+        LogTransaction.waitForDeletions();
+
+        try
+        {
+            // This should race with the asynchronous deletion of txn log files
+            // it should throw because we are violating the requirement that a transaction must
+            // finish before deleting files (i.e. releasing sstables)
+            LogAwareFileLister.getTemporaryFiles(dataFolder);
+            fail("Expected runtime exception");
+        }
+        catch(RuntimeException e)
+        {
+            //pass
+        }
+
+        logs.finish();
+    }
+
+    private static SSTableReader sstable(ColumnFamilyStore cfs, int generation, int size) throws IOException
+    {
+        Directories dir = new Directories(cfs.metadata);
+        Descriptor descriptor = new Descriptor(dir.getDirectoryForNewSSTables(), cfs.keyspace.getName(), cfs.getTableName(), generation);
+        Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
+        for (Component component : components)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                assertTrue(file.createNewFile());
+            try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
+            {
+                raf.setLength(size);
+            }
+        }
+
+        SegmentedFile dFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.DATA))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
+        SegmentedFile iFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
+
+        SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.emptyList());
+        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
+                                                 .finalizeMetadata(cfs.metadata.partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
+                                                 .get(MetadataType.STATS);
+        SSTableReader reader = SSTableReader.internalOpen(descriptor,
+                                                          components,
+                                                          cfs.metadata,
+                                                          dFile,
+                                                          iFile,
+                                                          MockSchema.indexSummary.sharedCopy(),
+                                                          new AlwaysPresentFilter(),
+                                                          1L,
+                                                          metadata,
+                                                          SSTableReader.OpenReason.NORMAL,
+                                                          header);
+        reader.first = reader.last = MockSchema.readerBounds(generation);
+        return reader;
+    }
+
+    private static void assertFiles(String dirPath, Set<String> expectedFiles)
+    {
+        assertFiles(dirPath, expectedFiles, false);
+    }
+
+    private static void assertFiles(String dirPath, Set<String> expectedFiles, boolean excludeNonExistingFiles)
+    {
+        LogTransaction.waitForDeletions();
+
+        File dir = new File(dirPath);
+        File[] files = dir.listFiles();
+        if (files != null)
+        {
+            for (File file : files)
+            {
+                if (file.isDirectory())
+                    continue;
+
+                String filePath = file.getPath();
+                assertTrue(filePath, expectedFiles.contains(filePath));
+                expectedFiles.remove(filePath);
+            }
+        }
+
+        if (excludeNonExistingFiles)
+        {
+            for (String filePath : expectedFiles)
+            {
+                File file = new File(filePath);
+                if (!file.exists())
+                    expectedFiles.remove(filePath);
+            }
+        }
+
+        assertTrue(expectedFiles.toString(), expectedFiles.isEmpty());
+    }
+
+    // Check either that a temporary file is expected to exist (in the existingFiles) or that
+    // it does not exist any longer (on Windows we need to check File.exists() because a list
+    // might return a file as existing even if it does not)
+    private static void assertFiles(Iterable<String> existingFiles, Set<File> temporaryFiles)
+    {
+        for (String filePath : existingFiles)
+        {
+            File file = new File(filePath);
+            assertTrue(filePath, temporaryFiles.contains(file));
+            temporaryFiles.remove(file);
+        }
+
+        for (File file : temporaryFiles)
+        {
+            if (!file.exists())
+                temporaryFiles.remove(file);
+        }
+
+        assertTrue(temporaryFiles.toString(), temporaryFiles.isEmpty());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
index 9eff1b1..a7ad156 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
@@ -20,13 +20,11 @@ package org.apache.cassandra.db.lifecycle;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.lang3.StringUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -36,7 +34,6 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
@@ -87,7 +84,7 @@ public class RealTransactionsTest extends SchemaLoader
         SSTableReader oldSSTable = getSSTable(cfs, 1);
         LifecycleTransaction txn = cfs.getTracker().tryModify(oldSSTable, OperationType.COMPACTION);
         SSTableReader newSSTable = replaceSSTable(cfs, txn, false);
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
 
         assertFiles(txn.log().getDataFolder(), new HashSet<>(newSSTable.getAllFilePaths()));
     }
@@ -102,7 +99,7 @@ public class RealTransactionsTest extends SchemaLoader
         LifecycleTransaction txn = cfs.getTracker().tryModify(oldSSTable, OperationType.COMPACTION);
 
         replaceSSTable(cfs, txn, true);
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
 
         assertFiles(txn.log().getDataFolder(), new HashSet<>(oldSSTable.getAllFilePaths()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index 0e36bb9..7b9b19c 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -30,7 +30,6 @@ import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -189,9 +188,9 @@ public class TrackerTest
     public void testDropSSTables()
     {
         testDropSSTables(false);
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
         testDropSSTables(true);
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
     }
 
     private void testDropSSTables(boolean invalidate)
@@ -214,7 +213,7 @@ public class TrackerTest
             else
             {
                 tracker.dropSSTables();
-                TransactionLog.waitForDeletions();
+                LogTransaction.waitForDeletions();
             }
             Assert.assertEquals(9, cfs.metric.totalDiskSpaceUsed.getCount());
             Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogTest.java
deleted file mode 100644
index 405d975..0000000
--- a/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogTest.java
+++ /dev/null
@@ -1,812 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.lifecycle;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;
-import static junit.framework.Assert.fail;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.SerializationHeader;
-import org.apache.cassandra.db.compaction.*;
-import org.apache.cassandra.io.sstable.*;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.io.sstable.metadata.MetadataType;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.BufferedSegmentedFile;
-import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.io.util.SegmentedFile;
-import org.apache.cassandra.utils.AlwaysPresentFilter;
-import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
-import org.apache.cassandra.utils.concurrent.Transactional;
-
-public class TransactionLogTest extends AbstractTransactionalTest
-{
-    private static final String KEYSPACE = "TransactionLogsTest";
-
-    @BeforeClass
-    public static void setUp()
-    {
-        MockSchema.cleanup();
-    }
-
-    protected AbstractTransactionalTest.TestableTransaction newTest() throws Exception
-    {
-        TransactionLog.waitForDeletions();
-        SSTableReader.resetTidying();
-        return new TxnTest();
-    }
-
-    private static final class TxnTest extends TestableTransaction
-    {
-        private final static class Transaction extends Transactional.AbstractTransactional implements Transactional
-        {
-            final ColumnFamilyStore cfs;
-            final TransactionLog txnLogs;
-            final SSTableReader sstableOld;
-            final SSTableReader sstableNew;
-            final TransactionLog.SSTableTidier tidier;
-
-            Transaction(ColumnFamilyStore cfs, TransactionLog txnLogs) throws IOException
-            {
-                this.cfs = cfs;
-                this.txnLogs = txnLogs;
-                this.sstableOld = sstable(cfs, 0, 128);
-                this.sstableNew = sstable(cfs, 1, 128);
-
-                assertNotNull(txnLogs);
-                assertNotNull(txnLogs.getId());
-                Assert.assertEquals(OperationType.COMPACTION, txnLogs.getType());
-
-                txnLogs.trackNew(sstableNew);
-                tidier = txnLogs.obsoleted(sstableOld);
-                assertNotNull(tidier);
-            }
-
-            protected Throwable doCommit(Throwable accumulate)
-            {
-                sstableOld.markObsolete(tidier);
-                sstableOld.selfRef().release();
-                TransactionLog.waitForDeletions();
-
-                Throwable ret = txnLogs.commit(accumulate);
-
-                sstableNew.selfRef().release();
-                return ret;
-            }
-
-            protected Throwable doAbort(Throwable accumulate)
-            {
-                tidier.abort();
-                TransactionLog.waitForDeletions();
-
-                Throwable ret = txnLogs.abort(accumulate);
-
-                sstableNew.selfRef().release();
-                sstableOld.selfRef().release();
-                return ret;
-            }
-
-            protected void doPrepare()
-            {
-                txnLogs.prepareToCommit();
-            }
-
-            void assertInProgress() throws Exception
-            {
-                assertFiles(txnLogs.getDataFolder(), Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths(),
-                                                                                      sstableOld.getAllFilePaths(),
-                                                                                      Collections.singleton(txnLogs.getData().getLogFile().file.getPath()))));
-            }
-
-            void assertPrepared() throws Exception
-            {
-            }
-
-            void assertAborted() throws Exception
-            {
-                assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
-            }
-
-            void assertCommitted() throws Exception
-            {
-                assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
-            }
-        }
-
-        final Transaction txn;
-
-        private TxnTest() throws IOException
-        {
-            this(MockSchema.newCFS(KEYSPACE));
-        }
-
-        private TxnTest(ColumnFamilyStore cfs) throws IOException
-        {
-            this(cfs, new TransactionLog(OperationType.COMPACTION, cfs.metadata));
-        }
-
-        private TxnTest(ColumnFamilyStore cfs, TransactionLog txnLogs) throws IOException
-        {
-            this(new Transaction(cfs, txnLogs));
-        }
-
-        private TxnTest(Transaction txn)
-        {
-            super(txn);
-            this.txn = txn;
-        }
-
-        protected void assertInProgress() throws Exception
-        {
-            txn.assertInProgress();
-        }
-
-        protected void assertPrepared() throws Exception
-        {
-            txn.assertPrepared();
-        }
-
-        protected void assertAborted() throws Exception
-        {
-            txn.assertAborted();
-        }
-
-        protected void assertCommitted() throws Exception
-        {
-            txn.assertCommitted();
-        }
-    }
-
-    @Test
-    public void testUntrack() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        // complete a transaction without keep the new files since they were untracked
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        transactionLog.untrackNew(sstableNew);
-
-        transactionLog.finish();
-
-        sstableNew.selfRef().release();
-        Thread.sleep(1);
-        TransactionLog.waitForDeletions();
-
-        assertFiles(transactionLog.getDataFolder(), Collections.<String>emptySet());
-    }
-
-    @Test
-    public void testCommitSameDesc() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld1 = sstable(cfs, 0, 128);
-        SSTableReader sstableOld2 = sstable(cfs, 0, 256);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-
-        sstableOld1.setReplaced();
-
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstableOld2);
-        assertNotNull(tidier);
-
-        transactionLog.finish();
-
-        sstableOld2.markObsolete(tidier);
-
-        sstableOld1.selfRef().release();
-        sstableOld2.selfRef().release();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
-
-        sstableNew.selfRef().release();
-    }
-
-    @Test
-    public void testCommitOnlyNew() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstable);
-        transactionLog.finish();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
-
-        sstable.selfRef().release();
-    }
-
-    @Test
-    public void testCommitOnlyOld() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstable);
-        assertNotNull(tidier);
-
-        transactionLog.finish();
-        sstable.markObsolete(tidier);
-        sstable.selfRef().release();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>());
-    }
-
-    @Test
-    public void testAbortOnlyNew() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstable);
-        transactionLog.abort();
-
-        sstable.selfRef().release();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>());
-    }
-
-    @Test
-    public void testAbortOnlyOld() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstable);
-        assertNotNull(tidier);
-
-        tidier.abort();
-        transactionLog.abort();
-
-        sstable.selfRef().release();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
-    }
-
-    @Test
-    public void testRemoveUnfinishedLeftovers_abort() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld = sstable(cfs, 0, 128);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        // simulate tracking sstables with a failed transaction (new log file NOT deleted)
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstableOld);
-
-        Set<File> tmpFiles = Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths().stream().map(File::new).collect(Collectors.toList()),
-                                                              Collections.singleton(transactionLog.getData().getLogFile().file)));
-
-        sstableNew.selfRef().release();
-        sstableOld.selfRef().release();
-
-        Assert.assertEquals(tmpFiles, TransactionLog.getTemporaryFiles(cfs.metadata, sstableNew.descriptor.directory));
-
-        // normally called at startup
-        TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-        // sstableOld should be only table left
-        Directories directories = new Directories(cfs.metadata);
-        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
-        assertEquals(1, sstables.size());
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
-
-        tidier.run();
-
-        // complete the transaction to avoid LEAK errors
-        transactionLog.close();
-    }
-
-    @Test
-    public void testRemoveUnfinishedLeftovers_commit() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld = sstable(cfs, 0, 128);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        // simulate tracking sstables with a committed transaction (new log file deleted)
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstableOld);
-
-        //Fake a commit
-        transactionLog.getData().getLogFile().commit();
-
-        Set<File> tmpFiles = Sets.newHashSet(Iterables.concat(sstableOld.getAllFilePaths().stream().map(p -> new File(p)).collect(Collectors.toList()),
-                                                              Collections.singleton(transactionLog.getData().getLogFile().file)));
-
-        sstableNew.selfRef().release();
-        sstableOld.selfRef().release();
-
-        Assert.assertEquals(tmpFiles, TransactionLog.getTemporaryFiles(cfs.metadata, sstableOld.descriptor.directory));
-
-        // normally called at startup
-        TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-        // sstableNew should be only table left
-        Directories directories = new Directories(cfs.metadata);
-        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
-        assertEquals(1, sstables.size());
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
-
-        tidier.run();
-
-        // complete the transaction to avoid LEAK errors
-        assertNull(transactionLog.complete(null));
-    }
-
-    @Test
-    public void testGetTemporaryFiles() throws IOException
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable1 = sstable(cfs, 0, 128);
-
-        File dataFolder = sstable1.descriptor.directory;
-
-        Set<File> tmpFiles = TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder);
-        assertNotNull(tmpFiles);
-        assertEquals(0, tmpFiles.size());
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.WRITE, cfs.metadata);
-        Directories directories = new Directories(cfs.metadata);
-
-        File[] beforeSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
-
-        SSTableReader sstable2 = sstable(cfs, 1, 128);
-        transactionLog.trackNew(sstable2);
-
-        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
-        assertEquals(2, sstables.size());
-
-        // this should contain sstable1, sstable2 and the transaction log file
-        File[] afterSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
-
-        int numNewFiles = afterSecondSSTable.length - beforeSecondSSTable.length;
-        assertEquals(numNewFiles - 1, sstable2.getAllFilePaths().size()); // new files except for transaction log file
-
-        tmpFiles = TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder);
-        assertNotNull(tmpFiles);
-        assertEquals(numNewFiles, tmpFiles.size());
-
-        File ssTable2DataFile = new File(sstable2.descriptor.filenameFor(Component.DATA));
-        File ssTable2IndexFile = new File(sstable2.descriptor.filenameFor(Component.PRIMARY_INDEX));
-
-        assertTrue(tmpFiles.contains(ssTable2DataFile));
-        assertTrue(tmpFiles.contains(ssTable2IndexFile));
-
-        List<File> files = directories.sstableLister(Directories.OnTxnErr.THROW).listFiles();
-        List<File> filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles();
-        assertNotNull(files);
-        assertNotNull(filesNoTmp);
-
-        assertTrue(files.contains(ssTable2DataFile));
-        assertTrue(files.contains(ssTable2IndexFile));
-
-        assertFalse(filesNoTmp.contains(ssTable2DataFile));
-        assertFalse(filesNoTmp.contains(ssTable2IndexFile));
-
-        transactionLog.finish();
-
-        //Now it should be empty since the transaction has finished
-        tmpFiles = TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder);
-        assertNotNull(tmpFiles);
-        assertEquals(0, tmpFiles.size());
-
-        filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles();
-        assertNotNull(filesNoTmp);
-        assertTrue(filesNoTmp.contains(ssTable2DataFile));
-        assertTrue(filesNoTmp.contains(ssTable2IndexFile));
-
-        sstable1.selfRef().release();
-        sstable2.selfRef().release();
-    }
-
-    @Test
-    public void testWrongChecksumLastLine() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake a commit with invalid checksum
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-                          },
-                          true);
-    }
-
-    @Test
-    public void testWrongChecksumSecondFromLastLine() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake two lines with invalid checksum
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("add:[ma-3-big,%d,4][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-                          },
-                          false);
-    }
-
-    @Test
-    public void testWrongChecksumLastLineMissingFile() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake a commit with invalid checksum and also delete one of the old files
-                              for (String filePath : s.getAllFilePaths())
-                              {
-                                  if (filePath.endsWith("Data.db"))
-                                  {
-                                      FileUtils.delete(filePath);
-                                      break;
-                                  }
-                              }
-
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-                          },
-                          false);
-    }
-
-    @Test
-    public void testWrongChecksumLastLineWrongRecordFormat() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake a commit with invalid checksum and a wrong record format (extra spaces)
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d ,0 ,0 ][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-                          },
-                          true);
-    }
-
-    @Test
-    public void testMissingChecksumLastLine() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          {
-                              // Fake a commit without a checksum
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0]",
-                                                             System.currentTimeMillis()));
-                          },
-                          true);
-    }
-
-    @Test
-    public void testMissingChecksumSecondFromLastLine() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake two lines without a checksum
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("add:[ma-3-big,%d,4]",
-                                                             System.currentTimeMillis()));
-
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0]",
-                                                             System.currentTimeMillis()));
-                          },
-                          false);
-    }
-
-    private void testCorruptRecord(BiConsumer<TransactionLog, SSTableReader> modifier, boolean isRecoverable) throws IOException
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld = sstable(cfs, 0, 128);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        File dataFolder = sstableOld.descriptor.directory;
-
-        // simulate tracking sstables with a committed transaction except the checksum will be wrong
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        transactionLog.obsoleted(sstableOld);
-
-        //Modify the transaction log in some way
-        modifier.accept(transactionLog, sstableOld);
-
-        String txnFilePath = transactionLog.getData().getLogFile().file.getPath();
-
-        assertNull(transactionLog.complete(null));
-
-        sstableOld.selfRef().release();
-        sstableNew.selfRef().release();
-
-        if (isRecoverable)
-        { // the corruption is recoverable, we assume there is a commit record
-
-            //This should return the old files and the tx log
-            assertFiles(Iterables.concat(sstableOld.getAllFilePaths(), Collections.singleton(txnFilePath)),
-                        TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder));
-
-            //This should remove old files
-            TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-            assertFiles(dataFolder.getPath(), Sets.newHashSet(sstableNew.getAllFilePaths()));
-        }
-        else
-        { // if an intermediate line was modified, we cannot tell,
-          // it should just throw and handle the exception with a log message
-
-            //This should not return any files
-            assertEquals(Collections.emptyList(), new TransactionLog.FileLister(dataFolder.toPath(),
-                                                                                (file, type) -> type != Directories.FileType.FINAL,
-                                                                                Directories.OnTxnErr.IGNORE).list());
-
-            try
-            {
-                //This should throw a RuntimeException
-                new TransactionLog.FileLister(dataFolder.toPath(),
-                                              (file, type) -> type != Directories.FileType.FINAL,
-                                              Directories.OnTxnErr.THROW).list();
-                fail("Expected exception");
-            }
-            catch (RuntimeException ex)
-            {
-                // pass
-                ex.printStackTrace();
-            }
-
-            //This should not remove any files
-            TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-            assertFiles(dataFolder.getPath(), Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths(),
-                                                                               sstableOld.getAllFilePaths(),
-                                                                               Collections.singleton(txnFilePath))),
-                        true);
-        }
-    }
-
-    @Test
-    public void testObsoletedDataFileUpdateTimeChanged() throws IOException
-    {
-        testObsoletedFilesChanged(sstable ->
-                                 {
-                                     // increase the modification time of the Data file
-                                     for (String filePath : sstable.getAllFilePaths())
-                                     {
-                                         if (filePath.endsWith("Data.db"))
-                                             assertTrue(new File(filePath).setLastModified(System.currentTimeMillis() + 60000)); //one minute later
-                                     }
-                                 });
-    }
-
-    private static void testObsoletedFilesChanged(Consumer<SSTableReader> modifier) throws IOException
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld = sstable(cfs, 0, 128);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        // simulate tracking sstables with a committed transaction except the checksum will be wrong
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        /*TransactionLog.SSTableTidier tidier =*/ transactionLog.obsoleted(sstableOld);
-
-        //modify the old sstable files
-        modifier.accept(sstableOld);
-
-        //Fake a commit
-        transactionLog.getData().getLogFile().commit();
-
-        //This should not remove the old files
-        TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-        assertFiles(transactionLog.getDataFolder(), Sets.newHashSet(Iterables.concat(
-                                                                                    sstableNew.getAllFilePaths(),
-                                                                                    sstableOld.getAllFilePaths(),
-                                                                                    Collections.singleton(transactionLog.getData().getLogFile().file.getPath()))));
-
-        sstableOld.selfRef().release();
-        sstableNew.selfRef().release();
-
-        // complete the transaction to avoid LEAK errors
-        assertNull(transactionLog.complete(null));
-
-        assertFiles(transactionLog.getDataFolder(), Sets.newHashSet(Iterables.concat(
-                                                                                    sstableNew.getAllFilePaths(),
-                                                                                    sstableOld.getAllFilePaths(),
-                                                                                    Collections.singleton(transactionLog.getData().getLogFile().file.getPath()))));
-    }
-
-    @Test
-    public void testGetTemporaryFilesSafeAfterObsoletion_1() throws Throwable
-    {
-        testGetTemporaryFilesSafeAfterObsoletion(true);
-    }
-
-    @Test
-    public void testGetTemporaryFilesSafeAfterObsoletion_2() throws Throwable
-    {
-        testGetTemporaryFilesSafeAfterObsoletion(false);
-    }
-
-    private void testGetTemporaryFilesSafeAfterObsoletion(boolean finishBefore) throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-        File dataFolder = sstable.descriptor.directory;
-
-        TransactionLog transactionLogs = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLogs);
-
-        TransactionLog.SSTableTidier tidier = transactionLogs.obsoleted(sstable);
-
-        if (finishBefore)
-            transactionLogs.finish();
-
-        sstable.markObsolete(tidier);
-        sstable.selfRef().release();
-
-        for (int i = 0; i < 100; i++)
-        {
-            // This should race with the asynchronous deletion of txn log files
-            // It doesn't matter what it returns but it should not throw
-            TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder);
-        }
-
-        if (!finishBefore)
-            transactionLogs.finish();
-    }
-
-    private static SSTableReader sstable(ColumnFamilyStore cfs, int generation, int size) throws IOException
-    {
-        Directories dir = new Directories(cfs.metadata);
-        Descriptor descriptor = new Descriptor(dir.getDirectoryForNewSSTables(), cfs.keyspace.getName(), cfs.getTableName(), generation);
-        Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
-        for (Component component : components)
-        {
-            File file = new File(descriptor.filenameFor(component));
-            if (!file.exists())
-                assertTrue(file.createNewFile());
-            try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
-            {
-                raf.setLength(size);
-            }
-        }
-
-        SegmentedFile dFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.DATA))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
-        SegmentedFile iFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
-
-        SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.emptyList());
-        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
-                                                 .finalizeMetadata(cfs.metadata.partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
-                                                 .get(MetadataType.STATS);
-        SSTableReader reader = SSTableReader.internalOpen(descriptor,
-                                                          components,
-                                                          cfs.metadata,
-                                                          dFile,
-                                                          iFile,
-                                                          MockSchema.indexSummary.sharedCopy(),
-                                                          new AlwaysPresentFilter(),
-                                                          1L,
-                                                          metadata,
-                                                          SSTableReader.OpenReason.NORMAL,
-                                                          header);
-        reader.first = reader.last = MockSchema.readerBounds(generation);
-        return reader;
-    }
-
-    private static void assertFiles(String dirPath, Set<String> expectedFiles)
-    {
-        assertFiles(dirPath, expectedFiles, false);
-    }
-
-    private static void assertFiles(String dirPath, Set<String> expectedFiles, boolean excludeNonExistingFiles)
-    {
-        TransactionLog.waitForDeletions();
-
-        File dir = new File(dirPath);
-        File[] files = dir.listFiles();
-        if (files != null)
-        {
-            for (File file : files)
-            {
-                if (file.isDirectory())
-                    continue;
-
-                String filePath = file.getPath();
-                assertTrue(filePath, expectedFiles.contains(filePath));
-                expectedFiles.remove(filePath);
-            }
-        }
-
-        if (excludeNonExistingFiles)
-        {
-            for (String filePath : expectedFiles)
-            {
-                File file = new File(filePath);
-                if (!file.exists())
-                    expectedFiles.remove(filePath);
-            }
-        }
-
-        assertTrue(expectedFiles.toString(), expectedFiles.isEmpty());
-    }
-
-    private static void assertFiles(Iterable<String> filePaths, Set<File> expectedFiles)
-    {
-        for (String filePath : filePaths)
-        {
-            File file = new File(filePath);
-            assertTrue(filePath, expectedFiles.contains(file));
-            expectedFiles.remove(file);
-        }
-
-        assertTrue(expectedFiles.isEmpty());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 1c61f51..942c7f9 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -44,7 +44,6 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.rows.EncodingStats;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.compaction.CompactionController;
@@ -109,7 +108,7 @@ public class SSTableRewriterTest extends SchemaLoader
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
         store.truncateBlocking();
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
     }
 
     @Test
@@ -145,7 +144,7 @@ public class SSTableRewriterTest extends SchemaLoader
             }
             writer.finish();
         }
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
         assertEquals(1, filecounts);
@@ -177,7 +176,7 @@ public class SSTableRewriterTest extends SchemaLoader
             }
             writer.finish();
         }
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
         assertEquals(1, filecounts);
@@ -232,7 +231,7 @@ public class SSTableRewriterTest extends SchemaLoader
             assertTrue(checked);
             writer.finish();
         }
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
         assertEquals(1, filecounts);
@@ -277,12 +276,12 @@ public class SSTableRewriterTest extends SchemaLoader
             // open till .abort() is called (via the builder)
             if (!FBUtilities.isWindows())
             {
-                TransactionLog.waitForDeletions();
+                LifecycleTransaction.waitForDeletions();
                 assertFileCounts(dir.list());
             }
             writer.abort();
             txn.abort();
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             int datafiles = assertFileCounts(dir.list());
             assertEquals(datafiles, 0);
             validateCFS(cfs);
@@ -328,7 +327,7 @@ public class SSTableRewriterTest extends SchemaLoader
             sstables = rewriter.finish();
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         long sum = 0;
         for (SSTableReader x : cfs.getLiveSSTables())
@@ -337,7 +336,7 @@ public class SSTableRewriterTest extends SchemaLoader
         assertEquals(startStorageMetricsLoad - sBytesOnDisk + sum, StorageMetrics.load.getCount());
         assertEquals(files, sstables.size());
         assertEquals(files, cfs.getLiveSSTables().size());
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         // tmplink and tmp files should be gone:
         assertEquals(sum, cfs.metric.totalDiskSpaceUsed.getCount());
@@ -382,7 +381,7 @@ public class SSTableRewriterTest extends SchemaLoader
 
         assertEquals(files, sstables.size());
         assertEquals(files, cfs.getLiveSSTables().size());
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         assertFileCounts(s.descriptor.directory.list());
         validateCFS(cfs);
@@ -519,7 +518,7 @@ public class SSTableRewriterTest extends SchemaLoader
             test.run(scanner, controller, s, cfs, rewriter, txn);
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         assertEquals(startSize, cfs.metric.liveDiskSpaceUsed.getCount());
         assertEquals(1, cfs.getLiveSSTables().size());
@@ -567,7 +566,7 @@ public class SSTableRewriterTest extends SchemaLoader
             }
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         assertEquals(files - 1, cfs.getLiveSSTables().size()); // we never wrote anything to the last file
         assertFileCounts(s.descriptor.directory.list());
@@ -609,7 +608,7 @@ public class SSTableRewriterTest extends SchemaLoader
             sstables = rewriter.finish();
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         assertFileCounts(s.descriptor.directory.list());
         validateCFS(cfs);
     }
@@ -650,7 +649,7 @@ public class SSTableRewriterTest extends SchemaLoader
         }
         assertEquals(files, sstables.size());
         assertEquals(files, cfs.getLiveSSTables().size());
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         assertFileCounts(s.descriptor.directory.list());
 
         validateCFS(cfs);
@@ -670,7 +669,7 @@ public class SSTableRewriterTest extends SchemaLoader
             splitter.split();
 
             assertFileCounts(s.descriptor.directory.list());
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
 
             for (File f : s.descriptor.directory.listFiles())
             {
@@ -746,7 +745,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 s.selfRef().release();
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         int filecount = assertFileCounts(s.descriptor.directory.list());
         assertEquals(filecount, 1);
@@ -825,7 +824,7 @@ public class SSTableRewriterTest extends SchemaLoader
             rewriter.finish();
         }
         validateKeys(keyspace);
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         validateCFS(cfs);
         truncate(cfs);
     }
@@ -923,7 +922,7 @@ public class SSTableRewriterTest extends SchemaLoader
     public static void truncate(ColumnFamilyStore cfs)
     {
         cfs.truncateBlocking();
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         Uninterruptibles.sleepUninterruptibly(10L, TimeUnit.MILLISECONDS);
         assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
         assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/schema/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/DefsTest.java b/test/unit/org/apache/cassandra/schema/DefsTest.java
index 5eed80f..ee73b2b 100644
--- a/test/unit/org/apache/cassandra/schema/DefsTest.java
+++ b/test/unit/org/apache/cassandra/schema/DefsTest.java
@@ -40,7 +40,7 @@ import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -546,7 +546,7 @@ public class DefsTest
 
         // check
         assertTrue(cfs.indexManager.listIndexes().isEmpty());
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
     }
 


[4/7] cassandra git commit: Handle non-atomic directory streams safely (CASSANDRA-10109)

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
new file mode 100644
index 0000000..a655fd8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@ -0,0 +1,823 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.*;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.BufferedSegmentedFile;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+public class LogTransactionTest extends AbstractTransactionalTest
+{
+    private static final String KEYSPACE = "TransactionLogsTest";
+
+    @BeforeClass
+    public static void setUp()
+    {
+        MockSchema.cleanup();
+    }
+
+    protected AbstractTransactionalTest.TestableTransaction newTest() throws Exception
+    {
+        LogTransaction.waitForDeletions();
+        SSTableReader.resetTidying();
+        return new TxnTest();
+    }
+
+    private static final class TxnTest extends TestableTransaction
+    {
+        private final static class Transaction extends Transactional.AbstractTransactional implements Transactional
+        {
+            final ColumnFamilyStore cfs;
+            final LogTransaction txnLogs;
+            final SSTableReader sstableOld;
+            final SSTableReader sstableNew;
+            final LogTransaction.SSTableTidier tidier;
+
+            Transaction(ColumnFamilyStore cfs, LogTransaction txnLogs) throws IOException
+            {
+                this.cfs = cfs;
+                this.txnLogs = txnLogs;
+                this.sstableOld = sstable(cfs, 0, 128);
+                this.sstableNew = sstable(cfs, 1, 128);
+
+                assertNotNull(txnLogs);
+                assertNotNull(txnLogs.getId());
+                Assert.assertEquals(OperationType.COMPACTION, txnLogs.getType());
+
+                txnLogs.trackNew(sstableNew);
+                tidier = txnLogs.obsoleted(sstableOld);
+                assertNotNull(tidier);
+            }
+
+            protected Throwable doCommit(Throwable accumulate)
+            {
+                sstableOld.markObsolete(tidier);
+                sstableOld.selfRef().release();
+                LogTransaction.waitForDeletions();
+
+                Throwable ret = txnLogs.commit(accumulate);
+
+                sstableNew.selfRef().release();
+                return ret;
+            }
+
+            protected Throwable doAbort(Throwable accumulate)
+            {
+                tidier.abort();
+                LogTransaction.waitForDeletions();
+
+                Throwable ret = txnLogs.abort(accumulate);
+
+                sstableNew.selfRef().release();
+                sstableOld.selfRef().release();
+                return ret;
+            }
+
+            protected void doPrepare()
+            {
+                txnLogs.prepareToCommit();
+            }
+
+            void assertInProgress() throws Exception
+            {
+                assertFiles(txnLogs.getDataFolder(), Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths(),
+                                                                                      sstableOld.getAllFilePaths(),
+                                                                                      Collections.singleton(txnLogs.getLogFile().file.getPath()))));
+            }
+
+            void assertPrepared() throws Exception
+            {
+            }
+
+            void assertAborted() throws Exception
+            {
+                assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
+            }
+
+            void assertCommitted() throws Exception
+            {
+                assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+            }
+        }
+
+        final Transaction txn;
+
+        private TxnTest() throws IOException
+        {
+            this(MockSchema.newCFS(KEYSPACE));
+        }
+
+        private TxnTest(ColumnFamilyStore cfs) throws IOException
+        {
+            this(cfs, new LogTransaction(OperationType.COMPACTION, cfs.metadata));
+        }
+
+        private TxnTest(ColumnFamilyStore cfs, LogTransaction txnLogs) throws IOException
+        {
+            this(new Transaction(cfs, txnLogs));
+        }
+
+        private TxnTest(Transaction txn)
+        {
+            super(txn);
+            this.txn = txn;
+        }
+
+        protected void assertInProgress() throws Exception
+        {
+            txn.assertInProgress();
+        }
+
+        protected void assertPrepared() throws Exception
+        {
+            txn.assertPrepared();
+        }
+
+        protected void assertAborted() throws Exception
+        {
+            txn.assertAborted();
+        }
+
+        protected void assertCommitted() throws Exception
+        {
+            txn.assertCommitted();
+        }
+    }
+
+    @Test
+    public void testUntrack() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        // complete a transaction without keep the new files since they were untracked
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        log.untrackNew(sstableNew);
+
+        log.finish();
+
+        sstableNew.selfRef().release();
+        Thread.sleep(1);
+        LogTransaction.waitForDeletions();
+
+        assertFiles(log.getDataFolder(), Collections.<String>emptySet());
+    }
+
+    @Test
+    public void testCommitSameDesc() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld1 = sstable(cfs, 0, 128);
+        SSTableReader sstableOld2 = sstable(cfs, 0, 256);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+
+        sstableOld1.setReplaced();
+
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld2);
+        assertNotNull(tidier);
+
+        log.finish();
+
+        sstableOld2.markObsolete(tidier);
+
+        sstableOld1.selfRef().release();
+        sstableOld2.selfRef().release();
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+
+        sstableNew.selfRef().release();
+    }
+
+    @Test
+    public void testCommitOnlyNew() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstable);
+        log.finish();
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
+
+        sstable.selfRef().release();
+    }
+
+    @Test
+    public void testCommitOnlyOld() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstable);
+        assertNotNull(tidier);
+
+        log.finish();
+        sstable.markObsolete(tidier);
+        sstable.selfRef().release();
+
+        assertFiles(log.getDataFolder(), new HashSet<>());
+    }
+
+    @Test
+    public void testAbortOnlyNew() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstable);
+        log.abort();
+
+        sstable.selfRef().release();
+
+        assertFiles(log.getDataFolder(), new HashSet<>());
+    }
+
+    @Test
+    public void testAbortOnlyOld() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstable);
+        assertNotNull(tidier);
+
+        tidier.abort();
+        log.abort();
+
+        sstable.selfRef().release();
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
+    }
+
+    @Test
+    public void testRemoveUnfinishedLeftovers_abort() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld = sstable(cfs, 0, 128);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        // simulate tracking sstables with a failed transaction (new log file NOT deleted)
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld);
+
+        Set<File> tmpFiles = sstableNew.getAllFilePaths().stream().map(File::new).collect(Collectors.toSet());
+
+        sstableNew.selfRef().release();
+        sstableOld.selfRef().release();
+
+        Assert.assertEquals(tmpFiles, LogAwareFileLister.getTemporaryFiles(sstableNew.descriptor.directory));
+
+        // normally called at startup
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+        // sstableOld should be only table left
+        Directories directories = new Directories(cfs.metadata);
+        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
+        assertEquals(1, sstables.size());
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
+
+        tidier.run();
+
+        // complete the transaction before releasing files
+        log.close();
+    }
+
+    @Test
+    public void testRemoveUnfinishedLeftovers_commit() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld = sstable(cfs, 0, 128);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        // simulate tracking sstables with a committed transaction (new log file deleted)
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld);
+
+        //Fake a commit
+        log.getLogFile().commit();
+
+        Set<File> tmpFiles = sstableOld.getAllFilePaths().stream().map(File::new).collect(Collectors.toSet());
+
+        sstableNew.selfRef().release();
+        sstableOld.selfRef().release();
+
+        Assert.assertEquals(tmpFiles, LogAwareFileLister.getTemporaryFiles(sstableOld.descriptor.directory));
+
+        // normally called at startup
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+        // sstableNew should be only table left
+        Directories directories = new Directories(cfs.metadata);
+        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
+        assertEquals(1, sstables.size());
+
+        assertFiles(log.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+
+        tidier.run();
+
+        // complete the transaction to avoid LEAK errors
+        assertNull(log.complete(null));
+    }
+
+    @Test
+    public void testGetTemporaryFiles() throws IOException
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable1 = sstable(cfs, 0, 128);
+
+        File dataFolder = sstable1.descriptor.directory;
+
+        Set<File> tmpFiles = LogAwareFileLister.getTemporaryFiles(dataFolder);
+        assertNotNull(tmpFiles);
+        assertEquals(0, tmpFiles.size());
+
+        try(LogTransaction log = new LogTransaction(OperationType.WRITE, cfs.metadata))
+        {
+            Directories directories = new Directories(cfs.metadata);
+
+            File[] beforeSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
+
+            SSTableReader sstable2 = sstable(cfs, 1, 128);
+            log.trackNew(sstable2);
+
+            Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
+            assertEquals(2, sstables.size());
+
+            // this should contain sstable1, sstable2 and the transaction log file
+            File[] afterSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
+
+            int numNewFiles = afterSecondSSTable.length - beforeSecondSSTable.length;
+            assertEquals(numNewFiles - 1, sstable2.getAllFilePaths().size()); // new files except for transaction log file
+
+            tmpFiles = LogAwareFileLister.getTemporaryFiles(dataFolder);
+            assertNotNull(tmpFiles);
+            assertEquals(numNewFiles - 1, tmpFiles.size());
+
+            File ssTable2DataFile = new File(sstable2.descriptor.filenameFor(Component.DATA));
+            File ssTable2IndexFile = new File(sstable2.descriptor.filenameFor(Component.PRIMARY_INDEX));
+
+            assertTrue(tmpFiles.contains(ssTable2DataFile));
+            assertTrue(tmpFiles.contains(ssTable2IndexFile));
+
+            List<File> files = directories.sstableLister(Directories.OnTxnErr.THROW).listFiles();
+            List<File> filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles();
+            assertNotNull(files);
+            assertNotNull(filesNoTmp);
+
+            assertTrue(files.contains(ssTable2DataFile));
+            assertTrue(files.contains(ssTable2IndexFile));
+
+            assertFalse(filesNoTmp.contains(ssTable2DataFile));
+            assertFalse(filesNoTmp.contains(ssTable2IndexFile));
+
+            log.finish();
+
+            //Now it should be empty since the transaction has finished
+            tmpFiles = LogAwareFileLister.getTemporaryFiles(dataFolder);
+            assertNotNull(tmpFiles);
+            assertEquals(0, tmpFiles.size());
+
+            filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles();
+            assertNotNull(filesNoTmp);
+            assertTrue(filesNoTmp.contains(ssTable2DataFile));
+            assertTrue(filesNoTmp.contains(ssTable2IndexFile));
+
+            sstable1.selfRef().release();
+            sstable2.selfRef().release();
+        }
+    }
+
+    @Test
+    public void testWrongChecksumLastLine() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake a commit with invalid checksum
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+                          },
+                          true);
+    }
+
+    @Test
+    public void testWrongChecksumSecondFromLastLine() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake two lines with invalid checksum
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("add:[ma-3-big,%d,4][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+                          },
+                          false);
+    }
+
+    @Test
+    public void testWrongChecksumLastLineMissingFile() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake a commit with invalid checksum and also delete one of the old files
+                              for (String filePath : s.getAllFilePaths())
+                              {
+                                  if (filePath.endsWith("Data.db"))
+                                  {
+                                      assertTrue(FileUtils.delete(filePath));
+                                      t.getLogFile().sync();
+                                      break;
+                                  }
+                              }
+
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+                          },
+                          false);
+    }
+
+    @Test
+    public void testWrongChecksumLastLineWrongRecordFormat() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake a commit with invalid checksum and a wrong record format (extra spaces)
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d ,0 ,0 ][%d]",
+                                                             System.currentTimeMillis(),
+                                                             12345678L));
+                          },
+                          true);
+    }
+
+    @Test
+    public void testMissingChecksumLastLine() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          {
+                              // Fake a commit without a checksum
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0]",
+                                                             System.currentTimeMillis()));
+                          },
+                          true);
+    }
+
+    @Test
+    public void testMissingChecksumSecondFromLastLine() throws IOException
+    {
+        testCorruptRecord((t, s) ->
+                          { // Fake two lines without a checksum
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("add:[ma-3-big,%d,4]",
+                                                             System.currentTimeMillis()));
+
+                              FileUtils.append(t.getLogFile().file,
+                                               String.format("commit:[%d,0,0]",
+                                                             System.currentTimeMillis()));
+                          },
+                          false);
+    }
+
+    private static void testCorruptRecord(BiConsumer<LogTransaction, SSTableReader> modifier, boolean isRecoverable) throws IOException
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld = sstable(cfs, 0, 128);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        File dataFolder = sstableOld.descriptor.directory;
+
+        // simulate tracking sstables with a committed transaction except the checksum will be wrong
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        log.obsoleted(sstableOld);
+
+        // Modify the transaction log or disk state for sstableOld
+        modifier.accept(log, sstableOld);
+
+        String txnFilePath = log.getLogFile().file.getPath();
+
+        assertNull(log.complete(null));
+
+        sstableOld.selfRef().release();
+        sstableNew.selfRef().release();
+
+        // The files on disk, for old files make sure to exclude the files that were deleted by the modifier
+        Set<String> newFiles = sstableNew.getAllFilePaths().stream().collect(Collectors.toSet());
+        Set<String> oldFiles = sstableOld.getAllFilePaths().stream().filter(p -> new File(p).exists()).collect(Collectors.toSet());
+
+        //This should filter as in progress since the last record is corrupt
+        assertFiles(newFiles, LogAwareFileLister.getTemporaryFiles(dataFolder));
+        assertFiles(oldFiles, LogAwareFileLister.getFinalFiles(dataFolder));
+
+        if (isRecoverable)
+        { // the corruption is recoverable but the commit record is unreadable so the transaction is still in progress
+
+            //This should remove new files
+            LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+            // make sure to exclude the old files that were deleted by the modifier
+            assertFiles(dataFolder.getPath(), oldFiles);
+        }
+        else
+        { // if an intermediate line was also modified, it should ignore the tx log file
+
+            //This should not remove any files
+            LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+            assertFiles(dataFolder.getPath(), Sets.newHashSet(Iterables.concat(newFiles,
+                                                                               oldFiles,
+                                                                               Collections.singleton(txnFilePath))));
+        }
+    }
+
+    @Test
+    public void testObsoletedDataFileUpdateTimeChanged() throws IOException
+    {
+        testObsoletedFilesChanged(sstable ->
+                                  {
+                                      // increase the modification time of the Data file
+                                      for (String filePath : sstable.getAllFilePaths())
+                                      {
+                                          if (filePath.endsWith("Data.db"))
+                                              assertTrue(new File(filePath).setLastModified(System.currentTimeMillis() + 60000)); //one minute later
+                                      }
+                                  });
+    }
+
+    private static void testObsoletedFilesChanged(Consumer<SSTableReader> modifier) throws IOException
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstableOld = sstable(cfs, 0, 128);
+        SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+        // simulate tracking sstables with a committed transaction except the checksum will be wrong
+        LogTransaction log = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(log);
+
+        log.trackNew(sstableNew);
+        /*TransactionLog.SSTableTidier tidier =*/ log.obsoleted(sstableOld);
+
+        //modify the old sstable files
+        modifier.accept(sstableOld);
+
+        //Fake a commit
+        log.getLogFile().commit();
+
+        //This should not remove the old files
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+
+        assertFiles(log.getDataFolder(), Sets.newHashSet(Iterables.concat(
+                                                                                    sstableNew.getAllFilePaths(),
+                                                                                    sstableOld.getAllFilePaths(),
+                                                                                    Collections.singleton(log.getLogFile().file.getPath()))));
+
+        sstableOld.selfRef().release();
+        sstableNew.selfRef().release();
+
+        // complete the transaction to avoid LEAK errors
+        assertNull(log.complete(null));
+
+        assertFiles(log.getDataFolder(), Sets.newHashSet(Iterables.concat(
+                                                                                    sstableNew.getAllFilePaths(),
+                                                                                    sstableOld.getAllFilePaths(),
+                                                                                    Collections.singleton(log.getLogFile().file.getPath()))));
+    }
+
+    @Test
+    public void testGetTemporaryFilesSafeAfterObsoletion() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+        File dataFolder = sstable.descriptor.directory;
+
+        LogTransaction logs = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(logs);
+
+        LogTransaction.SSTableTidier tidier = logs.obsoleted(sstable);
+
+        logs.finish();
+
+        sstable.markObsolete(tidier);
+        sstable.selfRef().release();
+
+        // This should race with the asynchronous deletion of txn log files
+        // It doesn't matter what it returns but it should not throw because the txn
+        // was completed before deleting files (i.e. releasing sstables)
+        for (int i = 0; i < 200; i++)
+            LogAwareFileLister.getTemporaryFiles(dataFolder);
+    }
+
+    @Test
+    public void testGetTemporaryFilesThrowsIfCompletingAfterObsoletion() throws Throwable
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+        SSTableReader sstable = sstable(cfs, 0, 128);
+        File dataFolder = sstable.descriptor.directory;
+
+        LogTransaction logs = new LogTransaction(OperationType.COMPACTION, cfs.metadata);
+        assertNotNull(logs);
+
+        LogTransaction.SSTableTidier tidier = logs.obsoleted(sstable);
+
+        sstable.markObsolete(tidier);
+        sstable.selfRef().release();
+
+        LogTransaction.waitForDeletions();
+
+        try
+        {
+            // This should race with the asynchronous deletion of txn log files
+            // it should throw because we are violating the requirement that a transaction must
+            // finish before deleting files (i.e. releasing sstables)
+            LogAwareFileLister.getTemporaryFiles(dataFolder);
+            fail("Expected runtime exception");
+        }
+        catch(RuntimeException e)
+        {
+            //pass
+        }
+
+        logs.finish();
+    }
+
+    private static SSTableReader sstable(ColumnFamilyStore cfs, int generation, int size) throws IOException
+    {
+        Directories dir = new Directories(cfs.metadata);
+        Descriptor descriptor = new Descriptor(dir.getDirectoryForNewSSTables(), cfs.keyspace.getName(), cfs.getTableName(), generation);
+        Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
+        for (Component component : components)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                assertTrue(file.createNewFile());
+            try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
+            {
+                raf.setLength(size);
+            }
+        }
+
+        SegmentedFile dFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.DATA))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
+        SegmentedFile iFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
+
+        SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.emptyList());
+        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
+                                                 .finalizeMetadata(cfs.metadata.partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
+                                                 .get(MetadataType.STATS);
+        SSTableReader reader = SSTableReader.internalOpen(descriptor,
+                                                          components,
+                                                          cfs.metadata,
+                                                          dFile,
+                                                          iFile,
+                                                          MockSchema.indexSummary.sharedCopy(),
+                                                          new AlwaysPresentFilter(),
+                                                          1L,
+                                                          metadata,
+                                                          SSTableReader.OpenReason.NORMAL,
+                                                          header);
+        reader.first = reader.last = MockSchema.readerBounds(generation);
+        return reader;
+    }
+
+    private static void assertFiles(String dirPath, Set<String> expectedFiles)
+    {
+        assertFiles(dirPath, expectedFiles, false);
+    }
+
+    private static void assertFiles(String dirPath, Set<String> expectedFiles, boolean excludeNonExistingFiles)
+    {
+        LogTransaction.waitForDeletions();
+
+        File dir = new File(dirPath);
+        File[] files = dir.listFiles();
+        if (files != null)
+        {
+            for (File file : files)
+            {
+                if (file.isDirectory())
+                    continue;
+
+                String filePath = file.getPath();
+                assertTrue(filePath, expectedFiles.contains(filePath));
+                expectedFiles.remove(filePath);
+            }
+        }
+
+        if (excludeNonExistingFiles)
+        {
+            for (String filePath : expectedFiles)
+            {
+                File file = new File(filePath);
+                if (!file.exists())
+                    expectedFiles.remove(filePath);
+            }
+        }
+
+        assertTrue(expectedFiles.toString(), expectedFiles.isEmpty());
+    }
+
+    // Check either that a temporary file is expected to exist (in the existingFiles) or that
+    // it does not exist any longer (on Windows we need to check File.exists() because a list
+    // might return a file as existing even if it does not)
+    private static void assertFiles(Iterable<String> existingFiles, Set<File> temporaryFiles)
+    {
+        for (String filePath : existingFiles)
+        {
+            File file = new File(filePath);
+            assertTrue(filePath, temporaryFiles.contains(file));
+            temporaryFiles.remove(file);
+        }
+
+        for (File file : temporaryFiles)
+        {
+            if (!file.exists())
+                temporaryFiles.remove(file);
+        }
+
+        assertTrue(temporaryFiles.toString(), temporaryFiles.isEmpty());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
index 9eff1b1..a7ad156 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
@@ -20,13 +20,11 @@ package org.apache.cassandra.db.lifecycle;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.lang3.StringUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -36,7 +34,6 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
@@ -87,7 +84,7 @@ public class RealTransactionsTest extends SchemaLoader
         SSTableReader oldSSTable = getSSTable(cfs, 1);
         LifecycleTransaction txn = cfs.getTracker().tryModify(oldSSTable, OperationType.COMPACTION);
         SSTableReader newSSTable = replaceSSTable(cfs, txn, false);
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
 
         assertFiles(txn.log().getDataFolder(), new HashSet<>(newSSTable.getAllFilePaths()));
     }
@@ -102,7 +99,7 @@ public class RealTransactionsTest extends SchemaLoader
         LifecycleTransaction txn = cfs.getTracker().tryModify(oldSSTable, OperationType.COMPACTION);
 
         replaceSSTable(cfs, txn, true);
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
 
         assertFiles(txn.log().getDataFolder(), new HashSet<>(oldSSTable.getAllFilePaths()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index 0e36bb9..7b9b19c 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -30,7 +30,6 @@ import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -189,9 +188,9 @@ public class TrackerTest
     public void testDropSSTables()
     {
         testDropSSTables(false);
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
         testDropSSTables(true);
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
     }
 
     private void testDropSSTables(boolean invalidate)
@@ -214,7 +213,7 @@ public class TrackerTest
             else
             {
                 tracker.dropSSTables();
-                TransactionLog.waitForDeletions();
+                LogTransaction.waitForDeletions();
             }
             Assert.assertEquals(9, cfs.metric.totalDiskSpaceUsed.getCount());
             Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogTest.java
deleted file mode 100644
index 405d975..0000000
--- a/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogTest.java
+++ /dev/null
@@ -1,812 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.lifecycle;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;
-import static junit.framework.Assert.fail;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.SerializationHeader;
-import org.apache.cassandra.db.compaction.*;
-import org.apache.cassandra.io.sstable.*;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.io.sstable.metadata.MetadataType;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.BufferedSegmentedFile;
-import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.io.util.SegmentedFile;
-import org.apache.cassandra.utils.AlwaysPresentFilter;
-import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
-import org.apache.cassandra.utils.concurrent.Transactional;
-
-public class TransactionLogTest extends AbstractTransactionalTest
-{
-    private static final String KEYSPACE = "TransactionLogsTest";
-
-    @BeforeClass
-    public static void setUp()
-    {
-        MockSchema.cleanup();
-    }
-
-    protected AbstractTransactionalTest.TestableTransaction newTest() throws Exception
-    {
-        TransactionLog.waitForDeletions();
-        SSTableReader.resetTidying();
-        return new TxnTest();
-    }
-
-    private static final class TxnTest extends TestableTransaction
-    {
-        private final static class Transaction extends Transactional.AbstractTransactional implements Transactional
-        {
-            final ColumnFamilyStore cfs;
-            final TransactionLog txnLogs;
-            final SSTableReader sstableOld;
-            final SSTableReader sstableNew;
-            final TransactionLog.SSTableTidier tidier;
-
-            Transaction(ColumnFamilyStore cfs, TransactionLog txnLogs) throws IOException
-            {
-                this.cfs = cfs;
-                this.txnLogs = txnLogs;
-                this.sstableOld = sstable(cfs, 0, 128);
-                this.sstableNew = sstable(cfs, 1, 128);
-
-                assertNotNull(txnLogs);
-                assertNotNull(txnLogs.getId());
-                Assert.assertEquals(OperationType.COMPACTION, txnLogs.getType());
-
-                txnLogs.trackNew(sstableNew);
-                tidier = txnLogs.obsoleted(sstableOld);
-                assertNotNull(tidier);
-            }
-
-            protected Throwable doCommit(Throwable accumulate)
-            {
-                sstableOld.markObsolete(tidier);
-                sstableOld.selfRef().release();
-                TransactionLog.waitForDeletions();
-
-                Throwable ret = txnLogs.commit(accumulate);
-
-                sstableNew.selfRef().release();
-                return ret;
-            }
-
-            protected Throwable doAbort(Throwable accumulate)
-            {
-                tidier.abort();
-                TransactionLog.waitForDeletions();
-
-                Throwable ret = txnLogs.abort(accumulate);
-
-                sstableNew.selfRef().release();
-                sstableOld.selfRef().release();
-                return ret;
-            }
-
-            protected void doPrepare()
-            {
-                txnLogs.prepareToCommit();
-            }
-
-            void assertInProgress() throws Exception
-            {
-                assertFiles(txnLogs.getDataFolder(), Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths(),
-                                                                                      sstableOld.getAllFilePaths(),
-                                                                                      Collections.singleton(txnLogs.getData().getLogFile().file.getPath()))));
-            }
-
-            void assertPrepared() throws Exception
-            {
-            }
-
-            void assertAborted() throws Exception
-            {
-                assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
-            }
-
-            void assertCommitted() throws Exception
-            {
-                assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
-            }
-        }
-
-        final Transaction txn;
-
-        private TxnTest() throws IOException
-        {
-            this(MockSchema.newCFS(KEYSPACE));
-        }
-
-        private TxnTest(ColumnFamilyStore cfs) throws IOException
-        {
-            this(cfs, new TransactionLog(OperationType.COMPACTION, cfs.metadata));
-        }
-
-        private TxnTest(ColumnFamilyStore cfs, TransactionLog txnLogs) throws IOException
-        {
-            this(new Transaction(cfs, txnLogs));
-        }
-
-        private TxnTest(Transaction txn)
-        {
-            super(txn);
-            this.txn = txn;
-        }
-
-        protected void assertInProgress() throws Exception
-        {
-            txn.assertInProgress();
-        }
-
-        protected void assertPrepared() throws Exception
-        {
-            txn.assertPrepared();
-        }
-
-        protected void assertAborted() throws Exception
-        {
-            txn.assertAborted();
-        }
-
-        protected void assertCommitted() throws Exception
-        {
-            txn.assertCommitted();
-        }
-    }
-
-    @Test
-    public void testUntrack() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        // complete a transaction without keep the new files since they were untracked
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        transactionLog.untrackNew(sstableNew);
-
-        transactionLog.finish();
-
-        sstableNew.selfRef().release();
-        Thread.sleep(1);
-        TransactionLog.waitForDeletions();
-
-        assertFiles(transactionLog.getDataFolder(), Collections.<String>emptySet());
-    }
-
-    @Test
-    public void testCommitSameDesc() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld1 = sstable(cfs, 0, 128);
-        SSTableReader sstableOld2 = sstable(cfs, 0, 256);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-
-        sstableOld1.setReplaced();
-
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstableOld2);
-        assertNotNull(tidier);
-
-        transactionLog.finish();
-
-        sstableOld2.markObsolete(tidier);
-
-        sstableOld1.selfRef().release();
-        sstableOld2.selfRef().release();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
-
-        sstableNew.selfRef().release();
-    }
-
-    @Test
-    public void testCommitOnlyNew() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstable);
-        transactionLog.finish();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
-
-        sstable.selfRef().release();
-    }
-
-    @Test
-    public void testCommitOnlyOld() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstable);
-        assertNotNull(tidier);
-
-        transactionLog.finish();
-        sstable.markObsolete(tidier);
-        sstable.selfRef().release();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>());
-    }
-
-    @Test
-    public void testAbortOnlyNew() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstable);
-        transactionLog.abort();
-
-        sstable.selfRef().release();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>());
-    }
-
-    @Test
-    public void testAbortOnlyOld() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstable);
-        assertNotNull(tidier);
-
-        tidier.abort();
-        transactionLog.abort();
-
-        sstable.selfRef().release();
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
-    }
-
-    @Test
-    public void testRemoveUnfinishedLeftovers_abort() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld = sstable(cfs, 0, 128);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        // simulate tracking sstables with a failed transaction (new log file NOT deleted)
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstableOld);
-
-        Set<File> tmpFiles = Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths().stream().map(File::new).collect(Collectors.toList()),
-                                                              Collections.singleton(transactionLog.getData().getLogFile().file)));
-
-        sstableNew.selfRef().release();
-        sstableOld.selfRef().release();
-
-        Assert.assertEquals(tmpFiles, TransactionLog.getTemporaryFiles(cfs.metadata, sstableNew.descriptor.directory));
-
-        // normally called at startup
-        TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-        // sstableOld should be only table left
-        Directories directories = new Directories(cfs.metadata);
-        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
-        assertEquals(1, sstables.size());
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
-
-        tidier.run();
-
-        // complete the transaction to avoid LEAK errors
-        transactionLog.close();
-    }
-
-    @Test
-    public void testRemoveUnfinishedLeftovers_commit() throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld = sstable(cfs, 0, 128);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        // simulate tracking sstables with a committed transaction (new log file deleted)
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        TransactionLog.SSTableTidier tidier = transactionLog.obsoleted(sstableOld);
-
-        //Fake a commit
-        transactionLog.getData().getLogFile().commit();
-
-        Set<File> tmpFiles = Sets.newHashSet(Iterables.concat(sstableOld.getAllFilePaths().stream().map(p -> new File(p)).collect(Collectors.toList()),
-                                                              Collections.singleton(transactionLog.getData().getLogFile().file)));
-
-        sstableNew.selfRef().release();
-        sstableOld.selfRef().release();
-
-        Assert.assertEquals(tmpFiles, TransactionLog.getTemporaryFiles(cfs.metadata, sstableOld.descriptor.directory));
-
-        // normally called at startup
-        TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-        // sstableNew should be only table left
-        Directories directories = new Directories(cfs.metadata);
-        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
-        assertEquals(1, sstables.size());
-
-        assertFiles(transactionLog.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
-
-        tidier.run();
-
-        // complete the transaction to avoid LEAK errors
-        assertNull(transactionLog.complete(null));
-    }
-
-    @Test
-    public void testGetTemporaryFiles() throws IOException
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable1 = sstable(cfs, 0, 128);
-
-        File dataFolder = sstable1.descriptor.directory;
-
-        Set<File> tmpFiles = TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder);
-        assertNotNull(tmpFiles);
-        assertEquals(0, tmpFiles.size());
-
-        TransactionLog transactionLog = new TransactionLog(OperationType.WRITE, cfs.metadata);
-        Directories directories = new Directories(cfs.metadata);
-
-        File[] beforeSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
-
-        SSTableReader sstable2 = sstable(cfs, 1, 128);
-        transactionLog.trackNew(sstable2);
-
-        Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
-        assertEquals(2, sstables.size());
-
-        // this should contain sstable1, sstable2 and the transaction log file
-        File[] afterSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
-
-        int numNewFiles = afterSecondSSTable.length - beforeSecondSSTable.length;
-        assertEquals(numNewFiles - 1, sstable2.getAllFilePaths().size()); // new files except for transaction log file
-
-        tmpFiles = TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder);
-        assertNotNull(tmpFiles);
-        assertEquals(numNewFiles, tmpFiles.size());
-
-        File ssTable2DataFile = new File(sstable2.descriptor.filenameFor(Component.DATA));
-        File ssTable2IndexFile = new File(sstable2.descriptor.filenameFor(Component.PRIMARY_INDEX));
-
-        assertTrue(tmpFiles.contains(ssTable2DataFile));
-        assertTrue(tmpFiles.contains(ssTable2IndexFile));
-
-        List<File> files = directories.sstableLister(Directories.OnTxnErr.THROW).listFiles();
-        List<File> filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles();
-        assertNotNull(files);
-        assertNotNull(filesNoTmp);
-
-        assertTrue(files.contains(ssTable2DataFile));
-        assertTrue(files.contains(ssTable2IndexFile));
-
-        assertFalse(filesNoTmp.contains(ssTable2DataFile));
-        assertFalse(filesNoTmp.contains(ssTable2IndexFile));
-
-        transactionLog.finish();
-
-        //Now it should be empty since the transaction has finished
-        tmpFiles = TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder);
-        assertNotNull(tmpFiles);
-        assertEquals(0, tmpFiles.size());
-
-        filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles();
-        assertNotNull(filesNoTmp);
-        assertTrue(filesNoTmp.contains(ssTable2DataFile));
-        assertTrue(filesNoTmp.contains(ssTable2IndexFile));
-
-        sstable1.selfRef().release();
-        sstable2.selfRef().release();
-    }
-
-    @Test
-    public void testWrongChecksumLastLine() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake a commit with invalid checksum
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-                          },
-                          true);
-    }
-
-    @Test
-    public void testWrongChecksumSecondFromLastLine() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake two lines with invalid checksum
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("add:[ma-3-big,%d,4][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-                          },
-                          false);
-    }
-
-    @Test
-    public void testWrongChecksumLastLineMissingFile() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake a commit with invalid checksum and also delete one of the old files
-                              for (String filePath : s.getAllFilePaths())
-                              {
-                                  if (filePath.endsWith("Data.db"))
-                                  {
-                                      FileUtils.delete(filePath);
-                                      break;
-                                  }
-                              }
-
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-                          },
-                          false);
-    }
-
-    @Test
-    public void testWrongChecksumLastLineWrongRecordFormat() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake a commit with invalid checksum and a wrong record format (extra spaces)
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d ,0 ,0 ][%d]",
-                                                             System.currentTimeMillis(),
-                                                             12345678L));
-                          },
-                          true);
-    }
-
-    @Test
-    public void testMissingChecksumLastLine() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          {
-                              // Fake a commit without a checksum
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0]",
-                                                             System.currentTimeMillis()));
-                          },
-                          true);
-    }
-
-    @Test
-    public void testMissingChecksumSecondFromLastLine() throws IOException
-    {
-        testCorruptRecord((t, s) ->
-                          { // Fake two lines without a checksum
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("add:[ma-3-big,%d,4]",
-                                                             System.currentTimeMillis()));
-
-                              FileUtils.append(t.getData().getLogFile().file,
-                                               String.format("commit:[%d,0,0]",
-                                                             System.currentTimeMillis()));
-                          },
-                          false);
-    }
-
-    private void testCorruptRecord(BiConsumer<TransactionLog, SSTableReader> modifier, boolean isRecoverable) throws IOException
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld = sstable(cfs, 0, 128);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        File dataFolder = sstableOld.descriptor.directory;
-
-        // simulate tracking sstables with a committed transaction except the checksum will be wrong
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        transactionLog.obsoleted(sstableOld);
-
-        //Modify the transaction log in some way
-        modifier.accept(transactionLog, sstableOld);
-
-        String txnFilePath = transactionLog.getData().getLogFile().file.getPath();
-
-        assertNull(transactionLog.complete(null));
-
-        sstableOld.selfRef().release();
-        sstableNew.selfRef().release();
-
-        if (isRecoverable)
-        { // the corruption is recoverable, we assume there is a commit record
-
-            //This should return the old files and the tx log
-            assertFiles(Iterables.concat(sstableOld.getAllFilePaths(), Collections.singleton(txnFilePath)),
-                        TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder));
-
-            //This should remove old files
-            TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-            assertFiles(dataFolder.getPath(), Sets.newHashSet(sstableNew.getAllFilePaths()));
-        }
-        else
-        { // if an intermediate line was modified, we cannot tell,
-          // it should just throw and handle the exception with a log message
-
-            //This should not return any files
-            assertEquals(Collections.emptyList(), new TransactionLog.FileLister(dataFolder.toPath(),
-                                                                                (file, type) -> type != Directories.FileType.FINAL,
-                                                                                Directories.OnTxnErr.IGNORE).list());
-
-            try
-            {
-                //This should throw a RuntimeException
-                new TransactionLog.FileLister(dataFolder.toPath(),
-                                              (file, type) -> type != Directories.FileType.FINAL,
-                                              Directories.OnTxnErr.THROW).list();
-                fail("Expected exception");
-            }
-            catch (RuntimeException ex)
-            {
-                // pass
-                ex.printStackTrace();
-            }
-
-            //This should not remove any files
-            TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-            assertFiles(dataFolder.getPath(), Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths(),
-                                                                               sstableOld.getAllFilePaths(),
-                                                                               Collections.singleton(txnFilePath))),
-                        true);
-        }
-    }
-
-    @Test
-    public void testObsoletedDataFileUpdateTimeChanged() throws IOException
-    {
-        testObsoletedFilesChanged(sstable ->
-                                 {
-                                     // increase the modification time of the Data file
-                                     for (String filePath : sstable.getAllFilePaths())
-                                     {
-                                         if (filePath.endsWith("Data.db"))
-                                             assertTrue(new File(filePath).setLastModified(System.currentTimeMillis() + 60000)); //one minute later
-                                     }
-                                 });
-    }
-
-    private static void testObsoletedFilesChanged(Consumer<SSTableReader> modifier) throws IOException
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstableOld = sstable(cfs, 0, 128);
-        SSTableReader sstableNew = sstable(cfs, 1, 128);
-
-        // simulate tracking sstables with a committed transaction except the checksum will be wrong
-        TransactionLog transactionLog = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLog);
-
-        transactionLog.trackNew(sstableNew);
-        /*TransactionLog.SSTableTidier tidier =*/ transactionLog.obsoleted(sstableOld);
-
-        //modify the old sstable files
-        modifier.accept(sstableOld);
-
-        //Fake a commit
-        transactionLog.getData().getLogFile().commit();
-
-        //This should not remove the old files
-        TransactionLog.removeUnfinishedLeftovers(cfs.metadata);
-
-        assertFiles(transactionLog.getDataFolder(), Sets.newHashSet(Iterables.concat(
-                                                                                    sstableNew.getAllFilePaths(),
-                                                                                    sstableOld.getAllFilePaths(),
-                                                                                    Collections.singleton(transactionLog.getData().getLogFile().file.getPath()))));
-
-        sstableOld.selfRef().release();
-        sstableNew.selfRef().release();
-
-        // complete the transaction to avoid LEAK errors
-        assertNull(transactionLog.complete(null));
-
-        assertFiles(transactionLog.getDataFolder(), Sets.newHashSet(Iterables.concat(
-                                                                                    sstableNew.getAllFilePaths(),
-                                                                                    sstableOld.getAllFilePaths(),
-                                                                                    Collections.singleton(transactionLog.getData().getLogFile().file.getPath()))));
-    }
-
-    @Test
-    public void testGetTemporaryFilesSafeAfterObsoletion_1() throws Throwable
-    {
-        testGetTemporaryFilesSafeAfterObsoletion(true);
-    }
-
-    @Test
-    public void testGetTemporaryFilesSafeAfterObsoletion_2() throws Throwable
-    {
-        testGetTemporaryFilesSafeAfterObsoletion(false);
-    }
-
-    private void testGetTemporaryFilesSafeAfterObsoletion(boolean finishBefore) throws Throwable
-    {
-        ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        SSTableReader sstable = sstable(cfs, 0, 128);
-        File dataFolder = sstable.descriptor.directory;
-
-        TransactionLog transactionLogs = new TransactionLog(OperationType.COMPACTION, cfs.metadata);
-        assertNotNull(transactionLogs);
-
-        TransactionLog.SSTableTidier tidier = transactionLogs.obsoleted(sstable);
-
-        if (finishBefore)
-            transactionLogs.finish();
-
-        sstable.markObsolete(tidier);
-        sstable.selfRef().release();
-
-        for (int i = 0; i < 100; i++)
-        {
-            // This should race with the asynchronous deletion of txn log files
-            // It doesn't matter what it returns but it should not throw
-            TransactionLog.getTemporaryFiles(cfs.metadata, dataFolder);
-        }
-
-        if (!finishBefore)
-            transactionLogs.finish();
-    }
-
-    private static SSTableReader sstable(ColumnFamilyStore cfs, int generation, int size) throws IOException
-    {
-        Directories dir = new Directories(cfs.metadata);
-        Descriptor descriptor = new Descriptor(dir.getDirectoryForNewSSTables(), cfs.keyspace.getName(), cfs.getTableName(), generation);
-        Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
-        for (Component component : components)
-        {
-            File file = new File(descriptor.filenameFor(component));
-            if (!file.exists())
-                assertTrue(file.createNewFile());
-            try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
-            {
-                raf.setLength(size);
-            }
-        }
-
-        SegmentedFile dFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.DATA))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
-        SegmentedFile iFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
-
-        SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.emptyList());
-        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
-                                                 .finalizeMetadata(cfs.metadata.partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
-                                                 .get(MetadataType.STATS);
-        SSTableReader reader = SSTableReader.internalOpen(descriptor,
-                                                          components,
-                                                          cfs.metadata,
-                                                          dFile,
-                                                          iFile,
-                                                          MockSchema.indexSummary.sharedCopy(),
-                                                          new AlwaysPresentFilter(),
-                                                          1L,
-                                                          metadata,
-                                                          SSTableReader.OpenReason.NORMAL,
-                                                          header);
-        reader.first = reader.last = MockSchema.readerBounds(generation);
-        return reader;
-    }
-
-    private static void assertFiles(String dirPath, Set<String> expectedFiles)
-    {
-        assertFiles(dirPath, expectedFiles, false);
-    }
-
-    private static void assertFiles(String dirPath, Set<String> expectedFiles, boolean excludeNonExistingFiles)
-    {
-        TransactionLog.waitForDeletions();
-
-        File dir = new File(dirPath);
-        File[] files = dir.listFiles();
-        if (files != null)
-        {
-            for (File file : files)
-            {
-                if (file.isDirectory())
-                    continue;
-
-                String filePath = file.getPath();
-                assertTrue(filePath, expectedFiles.contains(filePath));
-                expectedFiles.remove(filePath);
-            }
-        }
-
-        if (excludeNonExistingFiles)
-        {
-            for (String filePath : expectedFiles)
-            {
-                File file = new File(filePath);
-                if (!file.exists())
-                    expectedFiles.remove(filePath);
-            }
-        }
-
-        assertTrue(expectedFiles.toString(), expectedFiles.isEmpty());
-    }
-
-    private static void assertFiles(Iterable<String> filePaths, Set<File> expectedFiles)
-    {
-        for (String filePath : filePaths)
-        {
-            File file = new File(filePath);
-            assertTrue(filePath, expectedFiles.contains(file));
-            expectedFiles.remove(file);
-        }
-
-        assertTrue(expectedFiles.isEmpty());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 1c61f51..942c7f9 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -44,7 +44,6 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.rows.EncodingStats;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.compaction.CompactionController;
@@ -109,7 +108,7 @@ public class SSTableRewriterTest extends SchemaLoader
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
         store.truncateBlocking();
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
     }
 
     @Test
@@ -145,7 +144,7 @@ public class SSTableRewriterTest extends SchemaLoader
             }
             writer.finish();
         }
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
         assertEquals(1, filecounts);
@@ -177,7 +176,7 @@ public class SSTableRewriterTest extends SchemaLoader
             }
             writer.finish();
         }
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
         assertEquals(1, filecounts);
@@ -232,7 +231,7 @@ public class SSTableRewriterTest extends SchemaLoader
             assertTrue(checked);
             writer.finish();
         }
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
         assertEquals(1, filecounts);
@@ -277,12 +276,12 @@ public class SSTableRewriterTest extends SchemaLoader
             // open till .abort() is called (via the builder)
             if (!FBUtilities.isWindows())
             {
-                TransactionLog.waitForDeletions();
+                LifecycleTransaction.waitForDeletions();
                 assertFileCounts(dir.list());
             }
             writer.abort();
             txn.abort();
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             int datafiles = assertFileCounts(dir.list());
             assertEquals(datafiles, 0);
             validateCFS(cfs);
@@ -328,7 +327,7 @@ public class SSTableRewriterTest extends SchemaLoader
             sstables = rewriter.finish();
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         long sum = 0;
         for (SSTableReader x : cfs.getLiveSSTables())
@@ -337,7 +336,7 @@ public class SSTableRewriterTest extends SchemaLoader
         assertEquals(startStorageMetricsLoad - sBytesOnDisk + sum, StorageMetrics.load.getCount());
         assertEquals(files, sstables.size());
         assertEquals(files, cfs.getLiveSSTables().size());
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         // tmplink and tmp files should be gone:
         assertEquals(sum, cfs.metric.totalDiskSpaceUsed.getCount());
@@ -382,7 +381,7 @@ public class SSTableRewriterTest extends SchemaLoader
 
         assertEquals(files, sstables.size());
         assertEquals(files, cfs.getLiveSSTables().size());
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         assertFileCounts(s.descriptor.directory.list());
         validateCFS(cfs);
@@ -519,7 +518,7 @@ public class SSTableRewriterTest extends SchemaLoader
             test.run(scanner, controller, s, cfs, rewriter, txn);
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         assertEquals(startSize, cfs.metric.liveDiskSpaceUsed.getCount());
         assertEquals(1, cfs.getLiveSSTables().size());
@@ -567,7 +566,7 @@ public class SSTableRewriterTest extends SchemaLoader
             }
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         assertEquals(files - 1, cfs.getLiveSSTables().size()); // we never wrote anything to the last file
         assertFileCounts(s.descriptor.directory.list());
@@ -609,7 +608,7 @@ public class SSTableRewriterTest extends SchemaLoader
             sstables = rewriter.finish();
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         assertFileCounts(s.descriptor.directory.list());
         validateCFS(cfs);
     }
@@ -650,7 +649,7 @@ public class SSTableRewriterTest extends SchemaLoader
         }
         assertEquals(files, sstables.size());
         assertEquals(files, cfs.getLiveSSTables().size());
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         assertFileCounts(s.descriptor.directory.list());
 
         validateCFS(cfs);
@@ -670,7 +669,7 @@ public class SSTableRewriterTest extends SchemaLoader
             splitter.split();
 
             assertFileCounts(s.descriptor.directory.list());
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
 
             for (File f : s.descriptor.directory.listFiles())
             {
@@ -746,7 +745,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 s.selfRef().release();
         }
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         int filecount = assertFileCounts(s.descriptor.directory.list());
         assertEquals(filecount, 1);
@@ -825,7 +824,7 @@ public class SSTableRewriterTest extends SchemaLoader
             rewriter.finish();
         }
         validateKeys(keyspace);
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         validateCFS(cfs);
         truncate(cfs);
     }
@@ -923,7 +922,7 @@ public class SSTableRewriterTest extends SchemaLoader
     public static void truncate(ColumnFamilyStore cfs)
     {
         cfs.truncateBlocking();
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         Uninterruptibles.sleepUninterruptibly(10L, TimeUnit.MILLISECONDS);
         assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
         assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/schema/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/DefsTest.java b/test/unit/org/apache/cassandra/schema/DefsTest.java
index 5eed80f..ee73b2b 100644
--- a/test/unit/org/apache/cassandra/schema/DefsTest.java
+++ b/test/unit/org/apache/cassandra/schema/DefsTest.java
@@ -40,7 +40,7 @@ import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -546,7 +546,7 @@ public class DefsTest
 
         // check
         assertTrue(cfs.indexManager.listIndexes().isEmpty());
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
     }
 


[6/7] cassandra git commit: Handle non-atomic directory streams safely (CASSANDRA-10109)

Posted by be...@apache.org.
Handle non-atomic directory streams safely (CASSANDRA-10109)

This patch refactors the lifecycle transaction log and updates
the logic to be robust to non-atomic listings of directories

patch by stefania; reviewed by benedict for CASSANDRA-10109


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/351c7cac
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/351c7cac
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/351c7cac

Branch: refs/heads/trunk
Commit: 351c7caca311834f6c5bff08b0204943850214a9
Parents: 3818d30
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Thu Aug 27 14:09:45 2015 +0800
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Tue Sep 8 11:53:22 2015 +0100

----------------------------------------------------------------------
 .../apache/cassandra/db/ColumnFamilyStore.java  |    2 +-
 .../org/apache/cassandra/db/Directories.java    |    5 +-
 .../apache/cassandra/db/lifecycle/Helpers.java  |   12 +-
 .../db/lifecycle/LifecycleTransaction.java      |   64 +-
 .../db/lifecycle/LogAwareFileLister.java        |  196 +++
 .../apache/cassandra/db/lifecycle/LogFile.java  |  364 ++++++
 .../cassandra/db/lifecycle/LogRecord.java       |  208 ++++
 .../cassandra/db/lifecycle/LogTransaction.java  |  418 +++++++
 .../apache/cassandra/db/lifecycle/Tracker.java  |    4 +-
 .../cassandra/db/lifecycle/TransactionLog.java  | 1141 ------------------
 .../apache/cassandra/io/sstable/SSTable.java    |    1 -
 .../io/sstable/format/SSTableReader.java        |    5 +-
 .../org/apache/cassandra/io/util/FileUtils.java |   10 +-
 .../apache/cassandra/service/GCInspector.java   |    4 +-
 .../cassandra/service/StorageService.java       |    4 +-
 .../cassandra/tools/StandaloneScrubber.java     |    3 +-
 .../cassandra/tools/StandaloneSplitter.java     |    3 +-
 .../cassandra/tools/StandaloneUpgrader.java     |    3 +-
 .../org/apache/cassandra/utils/CLibrary.java    |    2 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |    5 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |    3 +-
 .../cassandra/db/lifecycle/HelpersTest.java     |    4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |    2 +-
 .../db/lifecycle/LogTransactionTest.java        |  823 +++++++++++++
 .../db/lifecycle/RealTransactionsTest.java      |    7 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |    7 +-
 .../db/lifecycle/TransactionLogTest.java        |  812 -------------
 .../io/sstable/SSTableRewriterTest.java         |   35 +-
 .../org/apache/cassandra/schema/DefsTest.java   |    4 +-
 29 files changed, 2109 insertions(+), 2042 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 096172d..979e8ba 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -510,7 +510,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         SystemKeyspace.removeTruncationRecord(metadata.cfId);
 
         data.dropSSTables();
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         indexManager.invalidateAllIndexesBlocking();
         materializedViewManager.invalidate();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index c17b1fd..c801952 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -32,6 +32,7 @@ import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BiFunction;
+import java.util.function.Consumer;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
@@ -550,7 +551,7 @@ public class Directories
         TEMPORARY,
 
         /** A transaction log file (contains information on final and temporary files). */
-        TXN_LOG
+        TXN_LOG;
     }
 
     /**
@@ -562,7 +563,7 @@ public class Directories
         /** Throw the exception */
         THROW,
 
-        /** Ignore the txn log file */
+        /** Ignore the problematic parts of the txn log file */
         IGNORE
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
index 98983c5..f9555f4 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
@@ -106,12 +106,12 @@ class Helpers
             assert !reader.isReplaced();
     }
 
-    static Throwable markObsolete(List<TransactionLog.Obsoletion> obsoletions, Throwable accumulate)
+    static Throwable markObsolete(List<LogTransaction.Obsoletion> obsoletions, Throwable accumulate)
     {
         if (obsoletions == null || obsoletions.isEmpty())
             return accumulate;
 
-        for (TransactionLog.Obsoletion obsoletion : obsoletions)
+        for (LogTransaction.Obsoletion obsoletion : obsoletions)
         {
             try
             {
@@ -125,13 +125,13 @@ class Helpers
         return accumulate;
     }
 
-    static Throwable prepareForObsoletion(Iterable<SSTableReader> readers, TransactionLog txnLogs, List<TransactionLog.Obsoletion> obsoletions, Throwable accumulate)
+    static Throwable prepareForObsoletion(Iterable<SSTableReader> readers, LogTransaction txnLogs, List<LogTransaction.Obsoletion> obsoletions, Throwable accumulate)
     {
         for (SSTableReader reader : readers)
         {
             try
             {
-                obsoletions.add(new TransactionLog.Obsoletion(reader, txnLogs.obsoleted(reader)));
+                obsoletions.add(new LogTransaction.Obsoletion(reader, txnLogs.obsoleted(reader)));
             }
             catch (Throwable t)
             {
@@ -141,12 +141,12 @@ class Helpers
         return accumulate;
     }
 
-    static Throwable abortObsoletion(List<TransactionLog.Obsoletion> obsoletions, Throwable accumulate)
+    static Throwable abortObsoletion(List<LogTransaction.Obsoletion> obsoletions, Throwable accumulate)
     {
         if (obsoletions == null || obsoletions.isEmpty())
             return accumulate;
 
-        for (TransactionLog.Obsoletion obsoletion : obsoletions)
+        for (LogTransaction.Obsoletion obsoletion : obsoletions)
         {
             try
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
index 520b229..59bbc7d 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -98,7 +98,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
 
     public final Tracker tracker;
     // The transaction logs keep track of new and old sstable files
-    private final TransactionLog transactionLog;
+    private final LogTransaction log;
     // the original readers this transaction was opened over, and that it guards
     // (no other transactions may operate over these readers concurrently)
     private final Set<SSTableReader> originals = new HashSet<>();
@@ -115,7 +115,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
     private final State staged = new State();
 
     // the tidier and their readers, to be used for marking readers obsoleted during a commit
-    private List<TransactionLog.Obsoletion> obsoletions;
+    private List<LogTransaction.Obsoletion> obsoletions;
 
     /**
      * construct a Transaction for use in an offline operation
@@ -143,7 +143,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
     public static LifecycleTransaction offline(OperationType operationType, CFMetaData metadata)
     {
         Tracker dummy = new Tracker(null, false);
-        return new LifecycleTransaction(dummy, new TransactionLog(operationType, metadata, dummy), Collections.emptyList());
+        return new LifecycleTransaction(dummy, new LogTransaction(operationType, metadata, dummy), Collections.emptyList());
     }
 
     /**
@@ -152,18 +152,18 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
     public static LifecycleTransaction offline(OperationType operationType, File operationFolder)
     {
         Tracker dummy = new Tracker(null, false);
-        return new LifecycleTransaction(dummy, new TransactionLog(operationType, operationFolder, dummy), Collections.emptyList());
+        return new LifecycleTransaction(dummy, new LogTransaction(operationType, operationFolder, dummy), Collections.emptyList());
     }
 
     LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable<SSTableReader> readers)
     {
-        this(tracker, new TransactionLog(operationType, getMetadata(tracker, readers), tracker), readers);
+        this(tracker, new LogTransaction(operationType, getMetadata(tracker, readers), tracker), readers);
     }
 
-    LifecycleTransaction(Tracker tracker, TransactionLog transactionLog, Iterable<SSTableReader> readers)
+    LifecycleTransaction(Tracker tracker, LogTransaction log, Iterable<SSTableReader> readers)
     {
         this.tracker = tracker;
-        this.transactionLog = transactionLog;
+        this.log = log;
         for (SSTableReader reader : readers)
         {
             originals.add(reader);
@@ -187,19 +187,19 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         return null;
     }
 
-    public TransactionLog log()
+    public LogTransaction log()
     {
-        return transactionLog;
+        return log;
     }
 
     public OperationType opType()
     {
-        return transactionLog.getType();
+        return log.getType();
     }
 
     public UUID opId()
     {
-        return transactionLog.getId();
+        return log.getId();
     }
 
     public void doPrepare()
@@ -212,8 +212,8 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
 
         // prepare for compaction obsolete readers as long as they were part of the original set
         // since those that are not original are early readers that share the same desc with the finals
-        maybeFail(prepareForObsoletion(filterIn(logged.obsolete, originals), transactionLog, obsoletions = new ArrayList<>(), null));
-        transactionLog.prepareToCommit();
+        maybeFail(prepareForObsoletion(filterIn(logged.obsolete, originals), log, obsoletions = new ArrayList<>(), null));
+        log.prepareToCommit();
     }
 
     /**
@@ -228,7 +228,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         maybeFail(accumulate);
 
         // transaction log commit failure means we must abort; safe commit is not possible
-        maybeFail(transactionLog.commit(null));
+        maybeFail(log.commit(null));
 
         // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done
         // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size
@@ -237,7 +237,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         accumulate = markObsolete(obsoletions, accumulate);
         accumulate = tracker.updateSizeTracking(logged.obsolete, logged.update, accumulate);
         accumulate = release(selfRefs(logged.obsolete), accumulate);
-        accumulate = tracker.notifySSTablesChanged(originals, logged.update, transactionLog.getType(), accumulate);
+        accumulate = tracker.notifySSTablesChanged(originals, logged.update, log.getType(), accumulate);
 
         return accumulate;
     }
@@ -253,16 +253,16 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         accumulate = abortObsoletion(obsoletions, accumulate);
 
         if (logged.isEmpty() && staged.isEmpty())
-            return transactionLog.abort(accumulate);
+            return log.abort(accumulate);
 
         // mark obsolete all readers that are not versions of those present in the original set
         Iterable<SSTableReader> obsolete = filterOut(concatUniq(staged.update, logged.update), originals);
         logger.debug("Obsoleting {}", obsolete);
 
-        accumulate = prepareForObsoletion(obsolete, transactionLog, obsoletions = new ArrayList<>(), accumulate);
+        accumulate = prepareForObsoletion(obsolete, log, obsoletions = new ArrayList<>(), accumulate);
         // it's safe to abort even if committed, see maybeFail in doCommit() above, in this case it will just report
         // a failure to abort, which is useful information to have for debug
-        accumulate = transactionLog.abort(accumulate);
+        accumulate = log.abort(accumulate);
         accumulate = markObsolete(obsoletions, accumulate);
 
         // replace all updated readers with a version restored to its original state
@@ -502,7 +502,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
             originals.remove(reader);
             marked.remove(reader);
         }
-        return new LifecycleTransaction(tracker, transactionLog.getType(), readers);
+        return new LifecycleTransaction(tracker, log.getType(), readers);
     }
 
     /**
@@ -535,17 +535,17 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
 
     public void trackNew(SSTable table)
     {
-        transactionLog.trackNew(table);
+        log.trackNew(table);
     }
 
     public void untrackNew(SSTable table)
     {
-        transactionLog.untrackNew(table);
+        log.untrackNew(table);
     }
 
     public static void removeUnfinishedLeftovers(CFMetaData metadata)
     {
-        TransactionLog.removeUnfinishedLeftovers(metadata);
+        LogTransaction.removeUnfinishedLeftovers(metadata);
     }
 
     /**
@@ -562,7 +562,25 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
      */
     public static List<File> getFiles(Path folder, BiFunction<File, Directories.FileType, Boolean> filter, Directories.OnTxnErr onTxnErr)
     {
-        return new TransactionLog.FileLister(folder, filter, onTxnErr).list();
+        return new LogAwareFileLister(folder, filter, onTxnErr).list();
+    }
+
+    /**
+     * Retry all deletions that failed the first time around (presumably b/c the sstable was still mmap'd.)
+     * Useful because there are times when we know GC has been invoked; also exposed as an mbean.
+     */
+    public static void rescheduleFailedDeletions()
+    {
+        LogTransaction.rescheduleFailedDeletions();
+    }
+
+    /**
+     * Deletions run on the nonPeriodicTasks executor, (both failedDeletions or global tidiers in SSTableReader)
+     * so by scheduling a new empty task and waiting for it we ensure any prior deletion has completed.
+     */
+    public static void waitForDeletions()
+    {
+        LogTransaction.waitForDeletions();
     }
 
     // a class representing the current state of the reader within this transaction, encoding the actions both logged

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java b/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
new file mode 100644
index 0000000..e086078
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
@@ -0,0 +1,196 @@
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.*;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.Directories;
+
+import static org.apache.cassandra.db.Directories.*;
+
+/**
+ * A class for listing files in a folder.
+ */
+final class LogAwareFileLister
+{
+    // The folder to scan
+    private final Path folder;
+
+    // The filter determines which files the client wants returned
+    private final BiFunction<File, FileType, Boolean> filter; //file, file type
+
+    // The behavior when we fail to list files
+    private final OnTxnErr onTxnErr;
+
+    // The unfiltered result
+    NavigableMap<File, Directories.FileType> files = new TreeMap<>();
+
+    @VisibleForTesting
+    LogAwareFileLister(Path folder, BiFunction<File, FileType, Boolean> filter, OnTxnErr onTxnErr)
+    {
+        this.folder = folder;
+        this.filter = filter;
+        this.onTxnErr = onTxnErr;
+    }
+
+    public List<File> list()
+    {
+        try
+        {
+            return innerList();
+        }
+        catch (Throwable t)
+        {
+            throw new RuntimeException(String.format("Failed to list files in %s", folder), t);
+        }
+    }
+
+    List<File> innerList() throws Throwable
+    {
+        list(Files.newDirectoryStream(folder))
+        .stream()
+        .filter((f) -> !LogFile.isLogFile(f))
+        .forEach((f) -> files.put(f, FileType.FINAL));
+
+        // Since many file systems are not atomic, we cannot be sure we have listed a consistent disk state
+        // (Linux would permit this, but for simplicity we keep our behaviour the same across platforms)
+        // so we must be careful to list txn log files AFTER every other file since these files are deleted last,
+        // after all other files are removed
+        list(Files.newDirectoryStream(folder, '*' + LogFile.EXT))
+        .stream()
+        .filter(LogFile::isLogFile)
+        .forEach(this::classifyFiles);
+
+        // Finally we apply the user filter before returning our result
+        return files.entrySet().stream()
+                    .filter((e) -> filter.apply(e.getKey(), e.getValue()))
+                    .map(Map.Entry::getKey)
+                    .collect(Collectors.toList());
+    }
+
+    static List<File> list(DirectoryStream<Path> stream) throws IOException
+    {
+        try
+        {
+            return StreamSupport.stream(stream.spliterator(), false)
+                                .map(Path::toFile)
+                                .filter((f) -> !f.isDirectory())
+                                .collect(Collectors.toList());
+        }
+        finally
+        {
+            stream.close();
+        }
+    }
+
+    /**
+     * We read txn log files, if we fail we throw only if the user has specified
+     * OnTxnErr.THROW, else we log an error and apply the txn log anyway
+     */
+    void classifyFiles(File txnFile)
+    {
+        LogFile txn = LogFile.make(txnFile, -1);
+        readTxnLog(txn);
+        classifyFiles(txn);
+        files.put(txnFile, FileType.TXN_LOG);
+    }
+
+    void readTxnLog(LogFile txn)
+    {
+        txn.readRecords();
+        if (!txn.verify() && onTxnErr == OnTxnErr.THROW)
+            throw new LogTransaction.CorruptTransactionLogException("Some records failed verification. See earlier in log for details.", txn);
+    }
+
+    void classifyFiles(LogFile txnFile)
+    {
+        Map<LogRecord, Set<File>> oldFiles = txnFile.getFilesOfType(files.navigableKeySet(), LogRecord.Type.REMOVE);
+        Map<LogRecord, Set<File>> newFiles = txnFile.getFilesOfType(files.navigableKeySet(), LogRecord.Type.ADD);
+
+        if (txnFile.completed())
+        { // last record present, filter regardless of disk status
+            setTemporary(txnFile, oldFiles.values(), newFiles.values());
+            return;
+        }
+
+        if (allFilesPresent(txnFile, oldFiles, newFiles))
+        {  // all files present, transaction is in progress, this will filter as aborted
+            setTemporary(txnFile, oldFiles.values(), newFiles.values());
+            return;
+        }
+
+        // some files are missing, we expect the txn file to either also be missing or completed, so check
+        // disk state again to resolve any previous races on non-atomic directory listing platforms
+
+        // if txn file also gone, then do nothing (all temporary should be gone, we could remove them if any)
+        if (!txnFile.exists())
+            return;
+
+        // otherwise read the file again to see if it is completed now
+        readTxnLog(txnFile);
+
+        if (txnFile.completed())
+        { // if after re-reading the txn is completed then filter accordingly
+            setTemporary(txnFile, oldFiles.values(), newFiles.values());
+            return;
+        }
+
+        // some files are missing and yet the txn is still there and not completed
+        // something must be wrong (see comment at the top of this file requiring txn to be
+        // completed before obsoleting or aborting sstables)
+        throw new RuntimeException(String.format("Failed to list directory files in %s, inconsistent disk state for transaction %s",
+                                                 folder,
+                                                 txnFile));
+    }
+
+    /** See if all files are present or if only the last record files are missing and it's a NEW record */
+    private static boolean allFilesPresent(LogFile txnFile, Map<LogRecord, Set<File>> oldFiles, Map<LogRecord, Set<File>> newFiles)
+    {
+        LogRecord lastRecord = txnFile.getLastRecord();
+        return !Stream.concat(oldFiles.entrySet().stream(),
+                              newFiles.entrySet().stream()
+                                      .filter((e) -> e.getKey() != lastRecord))
+                      .filter((e) -> e.getKey().numFiles > e.getValue().size())
+                      .findFirst().isPresent();
+    }
+
+    private void setTemporary(LogFile txnFile, Collection<Set<File>> oldFiles, Collection<Set<File>> newFiles)
+    {
+        Collection<Set<File>> temporary = txnFile.committed() ? oldFiles : newFiles;
+        temporary.stream()
+                 .flatMap(Set::stream)
+                 .forEach((f) -> this.files.put(f, FileType.TEMPORARY));
+    }
+
+    @VisibleForTesting
+    static Set<File> getTemporaryFiles(File folder)
+    {
+        return listFiles(folder, FileType.TEMPORARY);
+    }
+
+    @VisibleForTesting
+    static Set<File> getFinalFiles(File folder)
+    {
+        return listFiles(folder, FileType.FINAL);
+    }
+
+    @VisibleForTesting
+    static Set<File> listFiles(File folder, FileType ... types)
+    {
+        Collection<FileType> match = Arrays.asList(types);
+        return new LogAwareFileLister(folder.toPath(),
+                                      (file, type) -> match.contains(type),
+                                      OnTxnErr.IGNORE).list()
+                                                      .stream()
+                                                      .collect(Collectors.toSet());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogFile.java b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
new file mode 100644
index 0000000..c698722
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
@@ -0,0 +1,364 @@
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LogRecord.Type;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.big.BigFormat;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.CLibrary;
+
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * The transaction log file, which contains many records.
+ */
+final class LogFile
+{
+    private static final Logger logger = LoggerFactory.getLogger(LogFile.class);
+
+    static String EXT = ".log";
+    static char SEP = '_';
+    // cc_txn_opname_id.log (where cc is one of the sstable versions defined in BigVersion)
+    static Pattern FILE_REGEX = Pattern.compile(String.format("^(.{2})_txn_(.*)_(.*)%s$", EXT));
+
+    final File file;
+    final Set<LogRecord> records = new LinkedHashSet<>();
+    final OperationType opType;
+    final UUID id;
+    final File folder;
+    final int folderDescriptor;
+
+    static LogFile make(File logFile, int folderDescriptor)
+    {
+        Matcher matcher = LogFile.FILE_REGEX.matcher(logFile.getName());
+        assert matcher.matches() && matcher.groupCount() == 3;
+
+        // For now we don't need this but it is there in case we need to change
+        // file format later on, the version is the sstable version as defined in BigFormat
+        //String version = matcher.group(1);
+
+        OperationType operationType = OperationType.fromFileName(matcher.group(2));
+        UUID id = UUID.fromString(matcher.group(3));
+
+        return new LogFile(operationType, logFile.getParentFile(), folderDescriptor, id);
+    }
+
+    void sync()
+    {
+        if (folderDescriptor > 0)
+            CLibrary.trySync(folderDescriptor);
+    }
+
+    OperationType getType()
+    {
+        return opType;
+    }
+
+    UUID getId()
+    {
+        return id;
+    }
+
+    Throwable removeUnfinishedLeftovers(Throwable accumulate)
+    {
+        try
+        {
+            deleteRecords(committed() ? Type.REMOVE : Type.ADD);
+
+            // we sync the parent file descriptor between contents and log deletion
+            // to ensure there is a happens before edge between them
+            sync();
+
+            Files.delete(file.toPath());
+        }
+        catch (Throwable t)
+        {
+            accumulate = merge(accumulate, t);
+        }
+
+        return accumulate;
+    }
+
+    static boolean isLogFile(File file)
+    {
+        return LogFile.FILE_REGEX.matcher(file.getName()).matches();
+    }
+
+    LogFile(OperationType opType, File folder, int folderDescriptor, UUID id)
+    {
+        this.opType = opType;
+        this.id = id;
+        this.folder = folder;
+        this.file = new File(getFileName(folder, opType, id));
+        this.folderDescriptor = folderDescriptor;
+    }
+
+    public void readRecords()
+    {
+        assert records.isEmpty();
+        FileUtils.readLines(file).stream()
+                 .map(LogRecord::make)
+                 .forEach(records::add);
+    }
+
+    public boolean verify()
+    {
+        Optional<LogRecord> firstInvalid = records.stream()
+                                                  .filter(this::isInvalid)
+                                                  .findFirst();
+
+        if (!firstInvalid.isPresent())
+            return true;
+
+        LogRecord failedOn = firstInvalid.get();
+        if (getLastRecord() != failedOn)
+        {
+            logError(failedOn);
+            return false;
+        }
+
+        if (records.stream()
+                   .filter((r) -> r != failedOn)
+                   .filter(LogFile::isInvalidWithCorruptedLastRecord)
+                   .map(LogFile::logError)
+                   .findFirst().isPresent())
+        {
+            logError(failedOn);
+            return false;
+        }
+
+        // if only the last record is corrupt and all other records have matching files on disk, @see verifyRecord,
+        // then we simply exited whilst serializing the last record and we carry on
+        logger.warn(String.format("Last record of transaction %s is corrupt or incomplete [%s], but all previous records match state on disk; continuing",
+                                  id,
+                                  failedOn.error));
+        return true;
+    }
+
+    static LogRecord logError(LogRecord record)
+    {
+        logger.error("{}", record.error);
+        return record;
+    }
+
+    boolean isInvalid(LogRecord record)
+    {
+        if (!record.isValid())
+            return true;
+
+        if (record.type == Type.UNKNOWN)
+        {
+            record.error(String.format("Could not parse record [%s]", record));
+            return true;
+        }
+
+        if (record.checksum != record.computeChecksum())
+        {
+            record.error(String.format("Invalid checksum for sstable [%s], record [%s]: [%d] should have been [%d]",
+                                       record.relativeFilePath,
+                                       record,
+                                       record.checksum,
+                                       record.computeChecksum()));
+            return true;
+        }
+
+        if (record.type != Type.REMOVE)
+            return false;
+
+        List<File> files = record.getExistingFiles(folder);
+
+        // Paranoid sanity checks: we create another record by looking at the files as they are
+        // on disk right now and make sure the information still matches
+        record.onDiskRecord = LogRecord.make(record.type, files, 0, record.relativeFilePath);
+
+        if (record.updateTime != record.onDiskRecord.updateTime && record.onDiskRecord.numFiles > 0)
+        {
+            record.error(String.format("Unexpected files detected for sstable [%s], record [%s]: last update time [%tT] should have been [%tT]",
+                                       record.relativeFilePath,
+                                       record,
+                                       record.onDiskRecord.updateTime,
+                                       record.updateTime));
+            return true;
+        }
+
+        return false;
+    }
+
+    static boolean isInvalidWithCorruptedLastRecord(LogRecord record)
+    {
+        if (record.type == Type.REMOVE && record.onDiskRecord.numFiles < record.numFiles)
+        { // if we found a corruption in the last record, then we continue only if the number of files matches exactly for all previous records.
+            record.error(String.format("Incomplete fileset detected for sstable [%s], record [%s]: number of files [%d] should have been [%d]. Treating as unrecoverable due to corruption of the final record.",
+                         record.relativeFilePath,
+                         record.raw,
+                         record.onDiskRecord.numFiles,
+                         record.numFiles));
+            return true;
+        }
+        return false;
+    }
+
+    public void commit()
+    {
+        assert !completed() : "Already completed!";
+        addRecord(LogRecord.makeCommit(System.currentTimeMillis()));
+    }
+
+    public void abort()
+    {
+        assert !completed() : "Already completed!";
+        addRecord(LogRecord.makeAbort(System.currentTimeMillis()));
+    }
+
+    private boolean isLastRecordValidWithType(Type type)
+    {
+        LogRecord lastRecord = getLastRecord();
+        return lastRecord != null &&
+               lastRecord.type == type &&
+               !isInvalid(lastRecord);
+    }
+
+    public boolean committed()
+    {
+        return isLastRecordValidWithType(Type.COMMIT);
+    }
+
+    public boolean aborted()
+    {
+        return isLastRecordValidWithType(Type.ABORT);
+    }
+
+    public boolean completed()
+    {
+        return committed() || aborted();
+    }
+
+    public void add(Type type, SSTable table)
+    {
+        if (!addRecord(makeRecord(type, table)))
+            throw new IllegalStateException();
+    }
+
+    private LogRecord makeRecord(Type type, SSTable table)
+    {
+        assert type == Type.ADD || type == Type.REMOVE;
+        return LogRecord.make(type, folder, table);
+    }
+
+    private boolean addRecord(LogRecord record)
+    {
+        if (!records.add(record))
+            return false;
+
+        // we only checksum the records, not the checksums themselves
+        FileUtils.append(file, record.toString());
+        sync();
+        return true;
+    }
+
+    public void remove(Type type, SSTable table)
+    {
+        LogRecord record = makeRecord(type, table);
+
+        assert records.contains(record) : String.format("[%s] is not tracked by %s", record, file);
+
+        records.remove(record);
+        deleteRecord(record);
+    }
+
+    public boolean contains(Type type, SSTable table)
+    {
+        return records.contains(makeRecord(type, table));
+    }
+
+    public void deleteRecords(Type type)
+    {
+        assert file.exists() : String.format("Expected %s to exists", file);
+        records.stream()
+               .filter(type::matches)
+               .forEach(this::deleteRecord);
+        records.clear();
+    }
+
+    private void deleteRecord(LogRecord record)
+    {
+        List<File> files = record.getExistingFiles(folder);
+
+        // we sort the files in ascending update time order so that the last update time
+        // stays the same even if we only partially delete files
+        files.sort((f1, f2) -> Long.compare(f1.lastModified(), f2.lastModified()));
+
+        files.forEach(LogTransaction::delete);
+    }
+
+    public Map<LogRecord, Set<File>> getFilesOfType(NavigableSet<File> files, Type type)
+    {
+        Map<LogRecord, Set<File>> ret = new HashMap<>();
+
+        records.stream()
+               .filter(type::matches)
+               .filter(LogRecord::isValid)
+               .forEach((r) -> ret.put(r, getRecordFiles(files, r)));
+
+        return ret;
+    }
+
+    public LogRecord getLastRecord()
+    {
+        return Iterables.getLast(records, null);
+    }
+
+    private Set<File> getRecordFiles(NavigableSet<File> files, LogRecord record)
+    {
+        Set<File> ret = new HashSet<>();
+        for (File file : files.tailSet(new File(folder, record.relativeFilePath)))
+        {
+            if (!file.getName().startsWith(record.relativeFilePath))
+                break;
+            ret.add(file);
+        }
+        return ret;
+    }
+
+    public void delete()
+    {
+        LogTransaction.delete(file);
+    }
+
+    public boolean exists()
+    {
+        return file.exists();
+    }
+
+    @Override
+    public String toString()
+    {
+        return FileUtils.getRelativePath(folder.getPath(), file.getPath());
+    }
+
+    static String getFileName(File folder, OperationType opType, UUID id)
+    {
+        String fileName = StringUtils.join(BigFormat.latestVersion,
+                                           LogFile.SEP,
+                                           "txn",
+                                           LogFile.SEP,
+                                           opType.fileName,
+                                           LogFile.SEP,
+                                           id.toString(),
+                                           LogFile.EXT);
+        return StringUtils.join(folder, File.separator, fileName);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
new file mode 100644
index 0000000..0f0f3a2
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
@@ -0,0 +1,208 @@
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.zip.CRC32;
+
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * A log file record, each record is encoded in one line and has different
+ * content depending on the record type.
+ */
+final class LogRecord
+{
+    public enum Type
+    {
+        UNKNOWN, // a record that cannot be parsed
+        ADD,    // new files to be retained on commit
+        REMOVE, // old files to be retained on abort
+        COMMIT, // commit flag
+        ABORT;  // abort flag
+
+        public static Type fromPrefix(String prefix)
+        {
+            return valueOf(prefix.toUpperCase());
+        }
+
+        public boolean hasFile()
+        {
+            return this == Type.ADD || this == Type.REMOVE;
+        }
+
+        public boolean matches(LogRecord record)
+        {
+            return this == record.type;
+        }
+    }
+
+
+    public final Type type;
+    public final String relativeFilePath;
+    public final long updateTime;
+    public final int numFiles;
+    public final String raw;
+    public final long checksum;
+
+    public String error;
+    public LogRecord onDiskRecord;
+
+    // (add|remove|commit|abort):[*,*,*][checksum]
+    static Pattern REGEX = Pattern.compile("^(add|remove|commit|abort):\\[([^,]*),?([^,]*),?([^,]*)\\]\\[(\\d*)\\]$", Pattern.CASE_INSENSITIVE);
+
+    public static LogRecord make(String line)
+    {
+        try
+        {
+            Matcher matcher = REGEX.matcher(line);
+            if (!matcher.matches())
+                return new LogRecord(Type.UNKNOWN, "", 0, 0, 0, line)
+                       .error(String.format("Failed to parse [%s]", line));
+
+            Type type = Type.fromPrefix(matcher.group(1));
+            return new LogRecord(type, matcher.group(2), Long.valueOf(matcher.group(3)), Integer.valueOf(matcher.group(4)), Long.valueOf(matcher.group(5)), line);
+        }
+        catch (Throwable t)
+        {
+            return new LogRecord(Type.UNKNOWN, "", 0, 0, 0, line).error(t);
+        }
+    }
+
+    public static LogRecord makeCommit(long updateTime)
+    {
+        return new LogRecord(Type.COMMIT, "", updateTime, 0);
+    }
+
+    public static LogRecord makeAbort(long updateTime)
+    {
+        return new LogRecord(Type.ABORT, "", updateTime, 0);
+    }
+
+    public static LogRecord make(Type type, File parentFolder, SSTable table)
+    {
+        String relativePath = FileUtils.getRelativePath(parentFolder.getPath(), table.descriptor.baseFilename());
+        // why do we take the max of files.size() and table.getAllFilePaths().size()?
+        return make(type, getExistingFiles(parentFolder, relativePath), table.getAllFilePaths().size(), relativePath);
+    }
+
+    public static LogRecord make(Type type, List<File> files, int minFiles, String relativeFilePath)
+    {
+        long lastModified = files.stream().map(File::lastModified).reduce(0L, Long::max);
+        return new LogRecord(type, relativeFilePath, lastModified, Math.max(minFiles, files.size()));
+    }
+
+    private LogRecord(Type type,
+                      String relativeFilePath,
+                      long updateTime,
+                      int numFiles)
+    {
+        this(type, relativeFilePath, updateTime, numFiles, 0, null);
+    }
+
+    private LogRecord(Type type,
+                      String relativeFilePath,
+                      long updateTime,
+                      int numFiles,
+                      long checksum,
+                      String raw)
+    {
+        this.type = type;
+        this.relativeFilePath = type.hasFile() ? relativeFilePath : ""; // only meaningful for file records
+        this.updateTime = type == Type.REMOVE ? updateTime : 0; // only meaningful for old records
+        this.numFiles = type.hasFile() ? numFiles : 0; // only meaningful for file records
+        if (raw == null)
+        {
+            assert checksum == 0;
+            this.checksum = computeChecksum();
+            this.raw = format();
+        }
+        else
+        {
+            this.checksum = checksum;
+            this.raw = raw;
+        }
+
+        this.error = "";
+    }
+
+    public LogRecord error(Throwable t)
+    {
+        return error(t.getMessage());
+    }
+
+    public LogRecord error(String error)
+    {
+        this.error = error;
+        return this;
+    }
+
+    public boolean isValid()
+    {
+        return this.error.isEmpty();
+    }
+
+    private String format()
+    {
+        return String.format("%s:[%s,%d,%d][%d]", type.toString(), relativeFilePath, updateTime, numFiles, checksum);
+    }
+
+    public List<File> getExistingFiles(File folder)
+    {
+        if (!type.hasFile())
+            return Collections.emptyList();
+
+        return getExistingFiles(folder, relativeFilePath);
+    }
+
+    public static List<File> getExistingFiles(File parentFolder, String relativeFilePath)
+    {
+        return Arrays.asList(parentFolder.listFiles((dir, name) -> name.startsWith(relativeFilePath)));
+    }
+
+    @Override
+    public int hashCode()
+    {
+        // see comment in equals
+        return Objects.hash(type, relativeFilePath, error);
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (!(obj instanceof LogRecord))
+            return false;
+
+        final LogRecord other = (LogRecord)obj;
+
+        // we exclude on purpose checksum, update time and count as
+        // we don't want duplicated records that differ only by
+        // properties that might change on disk, especially COMMIT records,
+        // there should be only one regardless of update time
+        // however we must compare the error to make sure we have more than
+        // one UNKNOWN record, if we fail to parse more than one
+        return type == other.type &&
+               relativeFilePath.equals(other.relativeFilePath) &&
+               error.equals(other.error);
+    }
+
+    @Override
+    public String toString()
+    {
+        return raw;
+    }
+
+    long computeChecksum()
+    {
+        CRC32 crc32 = new CRC32();
+        crc32.update(relativeFilePath.getBytes(FileUtils.CHARSET));
+        crc32.update(type.toString().getBytes(FileUtils.CHARSET));
+        FBUtilities.updateChecksumInt(crc32, (int) updateTime);
+        FBUtilities.updateChecksumInt(crc32, (int) (updateTime >>> 32));
+        FBUtilities.updateChecksumInt(crc32, numFiles);
+        return crc32.getValue() & (Long.MAX_VALUE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
new file mode 100644
index 0000000..89d7beb
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@ -0,0 +1,418 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.util.*;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.Runnables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LogRecord.Type;
+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.SnapshotDeletingTask;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.RefCounted;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+/**
+ * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
+ * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
+ * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
+ * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
+ *
+ * IMPORTANT: The transaction must complete (commit or abort) before any temporary files are deleted, even though the
+ * txn log file itself will not be deleted until all tracked files are deleted. This is required by FileLister to ensure
+ * a consistent disk state. LifecycleTransaction ensures this requirement, so this class should really never be used
+ * outside of LT. @see FileLister.classifyFiles(TransactionData txn)
+ *
+ * A class that tracks sstable files involved in a transaction across sstables:
+ * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
+ *
+ * The transaction log file contains new and old sstables as follows:
+ *
+ * add:[sstable-2][CRC]
+ * remove:[sstable-1,max_update_time,num files][CRC]
+ *
+ * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
+ * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
+ * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
+ * and file sizes.
+ *
+ * Upon commit we add a final line to the log file:
+ *
+ * commit:[commit_time][CRC]
+ *
+ * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
+ * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
+ * was committed, vice-versa if the transaction was aborted.
+ *
+ * On start-up we look for any transaction log files and repeat the cleanup process described above.
+ *
+ * See CASSANDRA-7066 for full details.
+ */
+class LogTransaction extends Transactional.AbstractTransactional implements Transactional
+{
+    private static final Logger logger = LoggerFactory.getLogger(LogTransaction.class);
+
+    /**
+     * If the format of the lines in the transaction log is wrong or the checksum
+     * does not match, then we throw this exception.
+     */
+    public static final class CorruptTransactionLogException extends RuntimeException
+    {
+        public final LogFile file;
+
+        public CorruptTransactionLogException(String message, LogFile file)
+        {
+            super(message);
+            this.file = file;
+        }
+    }
+
+    private final Tracker tracker;
+    private final LogFile data;
+    private final Ref<LogTransaction> selfRef;
+    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
+    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
+    // Additionally, we need to make sure to delete the data file first, so on restart the others
+    // will be recognized as GCable.
+    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
+
+    LogTransaction(OperationType opType, CFMetaData metadata)
+    {
+        this(opType, metadata, null);
+    }
+
+    LogTransaction(OperationType opType, CFMetaData metadata, Tracker tracker)
+    {
+        this(opType, new Directories(metadata), tracker);
+    }
+
+    LogTransaction(OperationType opType, Directories directories, Tracker tracker)
+    {
+        this(opType, directories.getDirectoryForNewSSTables(), tracker);
+    }
+
+    LogTransaction(OperationType opType, File folder, Tracker tracker)
+    {
+        this.tracker = tracker;
+        int folderDescriptor = CLibrary.tryOpenDirectory(folder.getPath());
+        this.data = new LogFile(opType, folder, folderDescriptor, UUIDGen.getTimeUUID());
+        this.selfRef = new Ref<>(this, new TransactionTidier(data, folderDescriptor));
+
+        if (logger.isDebugEnabled())
+            logger.debug("Created transaction logs with id {}", data.id);
+    }
+
+    /**
+     * Track a reader as new.
+     **/
+    void trackNew(SSTable table)
+    {
+        data.add(Type.ADD, table);
+    }
+
+    /**
+     * Stop tracking a reader as new.
+     */
+    void untrackNew(SSTable table)
+    {
+        data.remove(Type.ADD, table);
+    }
+
+    /**
+     * Schedule a reader for deletion as soon as it is fully unreferenced.
+     */
+    SSTableTidier obsoleted(SSTableReader reader)
+    {
+        if (data.contains(Type.ADD, reader))
+        {
+            if (data.contains(Type.REMOVE, reader))
+                throw new IllegalArgumentException();
+
+            return new SSTableTidier(reader, true, this);
+        }
+
+        data.add(Type.REMOVE, reader);
+
+        if (tracker != null)
+            tracker.notifyDeleting(reader);
+
+        return new SSTableTidier(reader, false, this);
+    }
+
+    OperationType getType()
+    {
+        return data.getType();
+    }
+
+    UUID getId()
+    {
+        return data.getId();
+    }
+
+    @VisibleForTesting
+    String getDataFolder()
+    {
+        return data.folder.getPath();
+    }
+
+    @VisibleForTesting
+    LogFile getLogFile()
+    {
+        return data;
+    }
+
+    static void delete(File file)
+    {
+        try
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("Deleting {}", file);
+
+            Files.delete(file.toPath());
+        }
+        catch (NoSuchFileException e)
+        {
+            logger.error("Unable to delete {} as it does not exist", file);
+        }
+        catch (IOException e)
+        {
+            logger.error("Unable to delete {}", file, e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * The transaction tidier.
+     *
+     * When the transaction reference is fully released we try to delete all the obsolete files
+     * depending on the transaction result, as well as the transaction log file.
+     */
+    private static class TransactionTidier implements RefCounted.Tidy, Runnable
+    {
+        private final LogFile data;
+        private final int folderDescriptor;
+
+        TransactionTidier(LogFile data, int folderDescriptor)
+        {
+            this.data = data;
+            this.folderDescriptor = folderDescriptor;
+        }
+
+        public void tidy() throws Exception
+        {
+            run();
+        }
+
+        public String name()
+        {
+            return data.toString();
+        }
+
+        public void run()
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("Removing files for transaction {}", name());
+
+            assert data.completed() : "Expected a completed transaction: " + data;
+
+            Throwable err = data.removeUnfinishedLeftovers(null);
+
+            if (err != null)
+            {
+                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
+                failedDeletions.add(this);
+            }
+            else
+            {
+                if (logger.isDebugEnabled())
+                    logger.debug("Closing file transaction {}", name());
+                CLibrary.tryCloseFD(folderDescriptor);
+            }
+        }
+    }
+
+    static class Obsoletion
+    {
+        final SSTableReader reader;
+        final SSTableTidier tidier;
+
+        Obsoletion(SSTableReader reader, SSTableTidier tidier)
+        {
+            this.reader = reader;
+            this.tidier = tidier;
+        }
+    }
+
+    /**
+     * The SSTableReader tidier. When a reader is fully released and no longer referenced
+     * by any one, we run this. It keeps a reference to the parent transaction and releases
+     * it when done, so that the final transaction cleanup can run when all obsolete readers
+     * are released.
+     */
+    public static class SSTableTidier implements Runnable
+    {
+        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
+        private final Descriptor desc;
+        private final long sizeOnDisk;
+        private final Tracker tracker;
+        private final boolean wasNew;
+        private final Ref<LogTransaction> parentRef;
+
+        public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent)
+        {
+            this.desc = referent.descriptor;
+            this.sizeOnDisk = referent.bytesOnDisk();
+            this.tracker = parent.tracker;
+            this.wasNew = wasNew;
+            this.parentRef = parent.selfRef.tryRef();
+        }
+
+        public void run()
+        {
+            SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+
+            try
+            {
+                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
+                File datafile = new File(desc.filenameFor(Component.DATA));
+
+                delete(datafile);
+                // let the remainder be cleaned up by delete
+                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
+            }
+            catch (Throwable t)
+            {
+                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
+                failedDeletions.add(this);
+                return;
+            }
+
+            if (tracker != null && tracker.cfstore != null && !wasNew)
+                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
+
+            // release the referent to the parent so that the all transaction files can be released
+            parentRef.release();
+        }
+
+        public void abort()
+        {
+            parentRef.release();
+        }
+    }
+
+
+    static void rescheduleFailedDeletions()
+    {
+        Runnable task;
+        while ( null != (task = failedDeletions.poll()))
+            ScheduledExecutors.nonPeriodicTasks.submit(task);
+
+        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
+        SnapshotDeletingTask.rescheduleFailedTasks();
+    }
+
+    static void waitForDeletions()
+    {
+        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
+    }
+
+    @VisibleForTesting
+    Throwable complete(Throwable accumulate)
+    {
+        try
+        {
+            accumulate = selfRef.ensureReleased(accumulate);
+            return accumulate;
+        }
+        catch (Throwable t)
+        {
+            logger.error("Failed to complete file transaction {}", getId(), t);
+            return Throwables.merge(accumulate, t);
+        }
+    }
+
+    protected Throwable doCommit(Throwable accumulate)
+    {
+        data.commit();
+        return complete(accumulate);
+    }
+
+    protected Throwable doAbort(Throwable accumulate)
+    {
+        data.abort();
+        return complete(accumulate);
+    }
+
+    protected void doPrepare() { }
+
+    /**
+     * Called on startup to scan existing folders for any unfinished leftovers of
+     * operations that were ongoing when the process exited. Also called by the standalone
+     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
+     *
+     */
+    static void removeUnfinishedLeftovers(CFMetaData metadata)
+    {
+        for (File dir : new Directories(metadata).getCFDirectories())
+        {
+            int folderDescriptor = CLibrary.tryOpenDirectory(dir.getPath());
+            try
+            {
+                File[] logs = dir.listFiles(LogFile::isLogFile);
+
+                for (File log : logs)
+                {
+                    LogFile data = LogFile.make(log, folderDescriptor);
+                    data.readRecords();
+                    if (data.verify())
+                    {
+                        Throwable failure = data.removeUnfinishedLeftovers(null);
+                        logger.error("Failed to remove unfinished transaction leftovers for log {}", log, failure);
+                    }
+                    else
+                    {
+                        logger.error("Unexpected disk state: failed to read transaction log {}", log);
+                    }
+                }
+            }
+            finally
+            {
+                CLibrary.tryCloseFD(folderDescriptor);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index d028493..ffb71ee 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -224,7 +224,7 @@ public class Tracker
      */
     public Throwable dropSSTables(final Predicate<SSTableReader> remove, OperationType operationType, Throwable accumulate)
     {
-        try (TransactionLog txnLogs = new TransactionLog(operationType, cfstore.metadata, this))
+        try (LogTransaction txnLogs = new LogTransaction(operationType, cfstore.metadata, this))
         {
             Pair<View, View> result = apply(view -> {
                 Set<SSTableReader> toremove = copyOf(filter(view.sstables, and(remove, notIn(view.compacting))));
@@ -236,7 +236,7 @@ public class Tracker
 
             // It is important that any method accepting/returning a Throwable never throws an exception, and does its best
             // to complete the instructions given to it
-            List<TransactionLog.Obsoletion> obsoletions = new ArrayList<>();
+            List<LogTransaction.Obsoletion> obsoletions = new ArrayList<>();
             accumulate = prepareForObsoletion(removed, txnLogs, obsoletions, accumulate);
             try
             {


[7/7] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by be...@apache.org.
Merge branch 'cassandra-3.0' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/322e21ec
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/322e21ec
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/322e21ec

Branch: refs/heads/trunk
Commit: 322e21ec4ec4200d155ea5eb8e7545c446c3c726
Parents: fba356d 351c7ca
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Tue Sep 8 11:53:45 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Tue Sep 8 11:53:45 2015 +0100

----------------------------------------------------------------------
 .../apache/cassandra/db/ColumnFamilyStore.java  |    2 +-
 .../org/apache/cassandra/db/Directories.java    |    5 +-
 .../apache/cassandra/db/lifecycle/Helpers.java  |   12 +-
 .../db/lifecycle/LifecycleTransaction.java      |   64 +-
 .../db/lifecycle/LogAwareFileLister.java        |  196 +++
 .../apache/cassandra/db/lifecycle/LogFile.java  |  364 ++++++
 .../cassandra/db/lifecycle/LogRecord.java       |  208 ++++
 .../cassandra/db/lifecycle/LogTransaction.java  |  418 +++++++
 .../apache/cassandra/db/lifecycle/Tracker.java  |    4 +-
 .../cassandra/db/lifecycle/TransactionLog.java  | 1141 ------------------
 .../apache/cassandra/io/sstable/SSTable.java    |    1 -
 .../io/sstable/format/SSTableReader.java        |    5 +-
 .../org/apache/cassandra/io/util/FileUtils.java |   10 +-
 .../apache/cassandra/service/GCInspector.java   |    4 +-
 .../cassandra/service/StorageService.java       |    4 +-
 .../cassandra/tools/StandaloneScrubber.java     |    3 +-
 .../cassandra/tools/StandaloneSplitter.java     |    3 +-
 .../cassandra/tools/StandaloneUpgrader.java     |    3 +-
 .../org/apache/cassandra/utils/CLibrary.java    |    2 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |    5 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |    3 +-
 .../cassandra/db/lifecycle/HelpersTest.java     |    4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |    2 +-
 .../db/lifecycle/LogTransactionTest.java        |  823 +++++++++++++
 .../db/lifecycle/RealTransactionsTest.java      |    7 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |    7 +-
 .../db/lifecycle/TransactionLogTest.java        |  812 -------------
 .../io/sstable/SSTableRewriterTest.java         |   35 +-
 .../org/apache/cassandra/schema/DefsTest.java   |    4 +-
 29 files changed, 2109 insertions(+), 2042 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/322e21ec/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------


[5/7] cassandra git commit: Handle non-atomic directory streams safely (CASSANDRA-10109)

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java b/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
deleted file mode 100644
index 69de370..0000000
--- a/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
+++ /dev/null
@@ -1,1141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.lifecycle;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.NoSuchFileException;
-import java.nio.file.Path;
-import java.nio.file.SecureDirectoryStream;
-import java.util.*;
-import java.util.Objects;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.zip.CRC32;
-import java.util.zip.Checksum;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.Runnables;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.compaction.OperationType;
-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.SnapshotDeletingTask;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.format.big.BigFormat;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.concurrent.Ref;
-import org.apache.cassandra.utils.concurrent.RefCounted;
-import org.apache.cassandra.utils.concurrent.Transactional;
-
-import static org.apache.cassandra.db.Directories.OnTxnErr;
-import static org.apache.cassandra.db.Directories.FileType;
-import static org.apache.cassandra.utils.Throwables.maybeFail;
-import static org.apache.cassandra.utils.Throwables.merge;
-
-/**
- * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
- * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
- * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
- * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
- *
- * A class that tracks sstable files involved in a transaction across sstables:
- * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
- *
- * The transaction log file contains new and old sstables as follows:
- *
- * add:[sstable-2][CRC]
- * remove:[sstable-1,max_update_time,num files][CRC]
- *
- * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
- * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
- * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
- * and file sizes.
- *
- * Upon commit we add a final line to the log file:
- *
- * commit:[commit_time][CRC]
- *
- * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
- * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
- * was committed, vice-versa if the transaction was aborted.
- *
- * On start-up we look for any transaction log files and repeat the cleanup process described above.
- *
- * See CASSANDRA-7066 for full details.
- */
-public class TransactionLog extends Transactional.AbstractTransactional implements Transactional
-{
-    private static final Logger logger = LoggerFactory.getLogger(TransactionLog.class);
-    private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1L, TimeUnit.HOURS);
-
-    /**
-     * If the format of the lines in the transaction log is wrong or the checksum
-     * does not match, then we throw this exception.
-     */
-    public static final class CorruptTransactionLogException extends RuntimeException
-    {
-        public final TransactionFile file;
-
-        public CorruptTransactionLogException(String message, TransactionFile file)
-        {
-            super(message);
-            this.file = file;
-        }
-    }
-
-    public enum RecordType
-    {
-        ADD,    // new files to be retained on commit
-        REMOVE, // old files to be retained on abort
-        COMMIT, // commit flag
-        ABORT;  // abort flag
-        public static RecordType fromPrefix(String prefix)
-        {
-            return valueOf(prefix.toUpperCase());
-        }
-    }
-
-    /**
-     * A log file record, each record is encoded in one line and has different
-     * content depending on the record type.
-     */
-    final static class Record
-    {
-        public final RecordType type;
-        public final String relativeFilePath;
-        public final long updateTime;
-        public final int numFiles;
-        public final String record;
-
-        static String REGEX_STR = "^(add|remove|commit|abort):\\[([^,]*),?([^,]*),?([^,]*)\\]$";
-        static Pattern REGEX = Pattern.compile(REGEX_STR, Pattern.CASE_INSENSITIVE); // (add|remove|commit|abort):[*,*,*]
-
-        public static Record make(String record, boolean isLast)
-        {
-            try
-            {
-                Matcher matcher = REGEX.matcher(record);
-                if (!matcher.matches() || matcher.groupCount() != 4)
-                    throw new IllegalStateException(String.format("Invalid record \"%s\"", record));
-
-                RecordType type = RecordType.fromPrefix(matcher.group(1));
-                return new Record(type, matcher.group(2), Long.valueOf(matcher.group(3)), Integer.valueOf(matcher.group(4)), record);
-            }
-            catch (Throwable t)
-            {
-                if (!isLast)
-                    throw t;
-
-                int pos = record.indexOf(':');
-                if (pos <= 0)
-                    throw t;
-
-                RecordType recordType;
-                try
-                {
-                    recordType = RecordType.fromPrefix(record.substring(0, pos));
-                }
-                catch (Throwable ignore)
-                {
-                    throw t;
-                }
-
-                return new Record(recordType, "", 0, 0, record);
-
-            }
-        }
-
-        public static Record makeCommit(long updateTime)
-        {
-            return new Record(RecordType.COMMIT, "", updateTime, 0, "");
-        }
-
-        public static Record makeAbort(long updateTime)
-        {
-            return new Record(RecordType.ABORT, "", updateTime, 0, "");
-        }
-
-        public static Record makeNew(String relativeFilePath)
-        {
-            return new Record(RecordType.ADD, relativeFilePath, 0, 0, "");
-        }
-
-        public static Record makeOld(String parentFolder, String relativeFilePath)
-        {
-            return makeOld(getTrackedFiles(parentFolder, relativeFilePath), relativeFilePath);
-        }
-
-        public static Record makeOld(List<File> files, String relativeFilePath)
-        {
-            long lastModified = files.stream()
-                                     .mapToLong(File::lastModified)
-                                     .reduce(0L, Long::max);
-            return new Record(RecordType.REMOVE, relativeFilePath, lastModified, files.size(), "");
-        }
-
-        private Record(RecordType type,
-                       String relativeFilePath,
-                       long updateTime,
-                       int numFiles,
-                       String record)
-        {
-            this.type = type;
-            this.relativeFilePath = hasFilePath(type) ? relativeFilePath : ""; // only meaningful for some records
-            this.updateTime = type == RecordType.REMOVE ? updateTime : 0; // only meaningful for old records
-            this.numFiles = type == RecordType.REMOVE ? numFiles : 0; // only meaningful for old records
-            this.record = record.isEmpty() ? format() : record;
-        }
-
-        private static boolean hasFilePath(RecordType type)
-        {
-            return type == RecordType.ADD || type == RecordType.REMOVE;
-        }
-
-        private String format()
-        {
-            return String.format("%s:[%s,%d,%d]", type.toString(), relativeFilePath, updateTime, numFiles);
-        }
-
-        public byte[] getBytes()
-        {
-            return record.getBytes(FileUtils.CHARSET);
-        }
-
-        public boolean verify(String parentFolder, boolean lastRecordIsCorrupt)
-        {
-            if (type != RecordType.REMOVE)
-                return true;
-
-            List<File> files = getTrackedFiles(parentFolder);
-
-            // Paranoid sanity checks: we create another record by looking at the files as they are
-            // on disk right now and make sure the information still matches
-            Record currentRecord = Record.makeOld(files, relativeFilePath);
-            if (updateTime != currentRecord.updateTime && currentRecord.numFiles > 0)
-            {
-                logger.error("Unexpected files detected for sstable [{}], record [{}]: last update time [{}] should have been [{}]",
-                             relativeFilePath,
-                             record,
-                             new Date(currentRecord.updateTime),
-                             new Date(updateTime));
-                return false;
-            }
-
-            if (lastRecordIsCorrupt && currentRecord.numFiles < numFiles)
-            { // if we found a corruption in the last record, then we continue only if the number of files matches exactly.
-                logger.error("Unexpected files detected for sstable [{}], record [{}]: number of files [{}] should have been [{}]",
-                             relativeFilePath,
-                             record,
-                             currentRecord.numFiles,
-                             numFiles);
-                return false;
-            }
-
-            return true;
-        }
-
-        public List<File> getTrackedFiles(String parentFolder)
-        {
-            if (!hasFilePath(type))
-                return Collections.emptyList();
-
-            return getTrackedFiles(parentFolder, relativeFilePath);
-        }
-
-        public static List<File> getTrackedFiles(String parentFolder, String relativeFilePath)
-        {
-            return Arrays.asList(new File(parentFolder).listFiles((dir, name) -> name.startsWith(relativeFilePath)));
-        }
-
-        @Override
-        public int hashCode()
-        {
-            // see comment in equals
-            return Objects.hash(type, relativeFilePath);
-        }
-
-        @Override
-        public boolean equals(Object obj)
-        {
-            if (obj == null)
-                return false;
-
-            if (getClass() != obj.getClass())
-                return false;
-
-            final Record other = (Record)obj;
-
-            // we exclude on purpose checksum, update time and count as
-            // we don't want duplicated records that differ only by
-            // properties that might change on disk, especially COMMIT records,
-            // there should be only one regardless of update time
-            return type.equals(other.type) &&
-                   relativeFilePath.equals(other.relativeFilePath);
-        }
-
-        @Override
-        public String toString()
-        {
-            return record;
-        }
-    }
-
-    /**
-     * The transaction log file, which contains many records.
-     */
-    final static class TransactionFile
-    {
-        static String EXT = ".log";
-        static char SEP = '_';
-        // cc_txn_opname_id.log (where cc is one of the sstable versions defined in BigVersion)
-        static String FILE_REGEX_STR = String.format("^(.{2})_txn_(.*)_(.*)%s$", EXT);
-        static Pattern FILE_REGEX = Pattern.compile(FILE_REGEX_STR);
-        static String LINE_REGEX_STR = "^(.*)\\[(\\d*)\\]$"; // *[checksum]
-        static Pattern LINE_REGEX = Pattern.compile(LINE_REGEX_STR);
-
-        public final File file;
-        public final TransactionData parent;
-        public final Set<Record> records = new HashSet<>();
-        public final Checksum checksum = new CRC32();
-
-        public TransactionFile(TransactionData parent)
-        {
-            this.file = new File(parent.getFileName());
-            this.parent = parent;
-        }
-
-        public void readRecords()
-        {
-            records.clear();
-            checksum.reset();
-
-            Iterator<String> it = FileUtils.readLines(file).iterator();
-            while(it.hasNext())
-                records.add(readRecord(it.next(), !it.hasNext())); // JLS execution order is left-to-right
-
-            for (Record record : records)
-            {
-                if (!record.verify(parent.getFolder(), false))
-                    throw new CorruptTransactionLogException(String.format("Failed to verify transaction %s record [%s]: unexpected disk state, aborting", parent.getId(), record),
-                                                             this);
-            }
-        }
-
-        private Record readRecord(String line, boolean isLast)
-        {
-            Matcher matcher = LINE_REGEX.matcher(line);
-            if (!matcher.matches() || matcher.groupCount() != 2)
-            {
-                handleReadRecordError(String.format("cannot parse line \"%s\"", line), isLast);
-                return Record.make(line, isLast);
-            }
-
-            byte[] bytes = matcher.group(1).getBytes(FileUtils.CHARSET);
-            checksum.update(bytes, 0, bytes.length);
-
-            if (checksum.getValue() != Long.valueOf(matcher.group(2)))
-                handleReadRecordError(String.format("invalid line checksum %s for \"%s\"", matcher.group(2), line), isLast);
-
-            try
-            {
-                return Record.make(matcher.group(1), isLast);
-            }
-            catch (Throwable t)
-            {
-                throw new CorruptTransactionLogException(String.format("Cannot make record \"%s\": %s", line, t.getMessage()), this);
-            }
-        }
-
-        private void handleReadRecordError(String message, boolean isLast)
-        {
-            if (isLast)
-            {
-                for (Record record : records)
-                {
-                    if (!record.verify(parent.getFolder(), true))
-                        throw new CorruptTransactionLogException(String.format("Last record of transaction %s is corrupt [%s] and at least " +
-                                                                               "one previous record does not match state on disk, unexpected disk state, aborting",
-                                                                               parent.getId(), message),
-                                                                 this);
-                }
-
-                // if only the last record is corrupt and all other records have matching files on disk, @see verifyRecord,
-                // then we simply exited whilst serializing the last record and we carry on
-                logger.warn(String.format("Last record of transaction %s is corrupt or incomplete [%s], but all previous records match state on disk; continuing", parent.getId(), message));
-
-            }
-            else
-            {
-                throw new CorruptTransactionLogException(String.format("Non-last record of transaction %s is corrupt [%s], unexpected disk state, aborting", parent.getId(), message), this);
-            }
-        }
-
-        public void commit()
-        {
-            assert !completed() : "Already completed!";
-            addRecord(Record.makeCommit(System.currentTimeMillis()));
-        }
-
-        public void abort()
-        {
-            assert !completed() : "Already completed!";
-            addRecord(Record.makeAbort(System.currentTimeMillis()));
-        }
-
-        public boolean committed()
-        {
-            return records.contains(Record.makeCommit(0));
-        }
-
-        public boolean aborted()
-        {
-            return records.contains(Record.makeAbort(0));
-        }
-
-        public boolean completed()
-        {
-            return committed() || aborted();
-        }
-
-        public boolean add(RecordType type, SSTable table)
-        {
-            Record record = makeRecord(type, table);
-            if (records.contains(record))
-                return false;
-
-            addRecord(record);
-            return true;
-        }
-
-        private Record makeRecord(RecordType type, SSTable table)
-        {
-            String relativePath = FileUtils.getRelativePath(parent.getFolder(), table.descriptor.baseFilename());
-            if (type == RecordType.ADD)
-            {
-                return Record.makeNew(relativePath);
-            }
-            else if (type == RecordType.REMOVE)
-            {
-                return Record.makeOld(parent.getFolder(), relativePath);
-            }
-            else
-            {
-                throw new AssertionError("Invalid record type " + type);
-            }
-        }
-
-        private void addRecord(Record record)
-        {
-            // we only checksum the records, not the checksums themselves
-            byte[] bytes = record.getBytes();
-            checksum.update(bytes, 0, bytes.length);
-
-            records.add(record);
-            FileUtils.append(file, String.format("%s[%d]", record, checksum.getValue()));
-
-            parent.sync();
-        }
-
-        public void remove(RecordType type, SSTable table)
-        {
-            Record record = makeRecord(type, table);
-
-            assert records.contains(record) : String.format("[%s] is not tracked by %s", record, file);
-
-            records.remove(record);
-            deleteRecord(record);
-        }
-
-        public boolean contains(RecordType type, SSTable table)
-        {
-            return records.contains(makeRecord(type, table));
-        }
-
-        public void deleteRecords(RecordType type)
-        {
-            assert file.exists() : String.format("Expected %s to exists", file);
-            records.stream()
-                   .filter((r) -> r.type == type)
-                   .forEach(this::deleteRecord);
-            records.clear();
-        }
-
-        private void deleteRecord(Record record)
-        {
-            List<File> files = record.getTrackedFiles(parent.getFolder());
-            if (files.isEmpty())
-                return; // Files no longer exist, nothing to do
-
-            // we sort the files in ascending update time order so that the last update time
-            // stays the same even if we only partially delete files
-            files.sort((f1, f2) -> Long.compare(f1.lastModified(), f2.lastModified()));
-
-            files.forEach(TransactionLog::delete);
-        }
-
-        public Set<File> getTrackedFiles(RecordType type)
-        {
-            return records.stream()
-                          .filter((r) -> r.type == type)
-                          .map((r) -> r.getTrackedFiles(parent.getFolder()))
-                          .flatMap(List::stream)
-                          .collect(Collectors.toSet());
-        }
-
-        public void delete()
-        {
-            TransactionLog.delete(file);
-        }
-
-        public boolean exists()
-        {
-            return file.exists();
-        }
-
-        @Override
-        public String toString()
-        {
-            return FileUtils.getRelativePath(parent.getFolder(), FileUtils.getCanonicalPath(file));
-        }
-    }
-
-    /**
-     * We split the transaction data from TransactionLog that implements the behavior
-     * because we need to reconstruct any left-overs and clean them up, as well as work
-     * out which files are temporary. So for these cases we don't want the full
-     * transactional behavior, plus it's handy for the TransactionTidier.
-     */
-    final static class TransactionData implements AutoCloseable
-    {
-        private final OperationType opType;
-        private final UUID id;
-        private final File folder;
-        private final TransactionFile file;
-        private int folderDescriptor;
-
-        static TransactionData make(File logFile)
-        {
-            Matcher matcher = TransactionFile.FILE_REGEX.matcher(logFile.getName());
-            assert matcher.matches() && matcher.groupCount() == 3;
-
-            // For now we don't need this but it is there in case we need to change
-            // file format later on, the version is the sstable version as defined in BigFormat
-            //String version = matcher.group(1);
-
-            OperationType operationType = OperationType.fromFileName(matcher.group(2));
-            UUID id = UUID.fromString(matcher.group(3));
-
-            return new TransactionData(operationType, logFile.getParentFile(), id);
-        }
-
-        TransactionData(OperationType opType, File folder, UUID id)
-        {
-            this.opType = opType;
-            this.id = id;
-            this.folder = folder;
-            this.file = new TransactionFile(this);
-            this.folderDescriptor = CLibrary.tryOpenDirectory(folder.getPath());
-        }
-
-        public Throwable readLogFile(Throwable accumulate)
-        {
-            try
-            {
-                file.readRecords();
-            }
-            catch (Throwable t)
-            {
-                accumulate = merge(accumulate, t);
-            }
-
-            return accumulate;
-        }
-
-        public void close()
-        {
-            if (folderDescriptor > 0)
-            {
-                CLibrary.tryCloseFD(folderDescriptor);
-                folderDescriptor = -1;
-            }
-        }
-
-        void sync()
-        {
-            if (folderDescriptor > 0)
-                CLibrary.trySync(folderDescriptor);
-        }
-
-        OperationType getType()
-        {
-            return opType;
-        }
-
-        UUID getId()
-        {
-            return id;
-        }
-
-        boolean completed()
-        {
-            return  file.completed();
-        }
-
-        Throwable removeUnfinishedLeftovers(Throwable accumulate)
-        {
-            try
-            {
-                if (file.committed())
-                    file.deleteRecords(RecordType.REMOVE);
-                else
-                    file.deleteRecords(RecordType.ADD);
-
-                // we sync the parent file descriptor between contents and log deletion
-                // to ensure there is a happens before edge between them
-                sync();
-
-                file.delete();
-            }
-            catch (Throwable t)
-            {
-                accumulate = merge(accumulate, t);
-            }
-
-            return accumulate;
-        }
-
-        Set<File> getTemporaryFiles()
-        {
-            sync();
-
-            if (!file.exists())
-                return Collections.emptySet();
-
-            if (file.committed())
-                return file.getTrackedFiles(RecordType.REMOVE);
-            else
-                return file.getTrackedFiles(RecordType.ADD);
-        }
-
-        String getFileName()
-        {
-            String fileName = StringUtils.join(BigFormat.latestVersion,
-                                               TransactionFile.SEP,
-                                               "txn",
-                                               TransactionFile.SEP,
-                                               opType.fileName,
-                                               TransactionFile.SEP,
-                                               id.toString(),
-                                               TransactionFile.EXT);
-            return StringUtils.join(folder, File.separator, fileName);
-        }
-
-        String getFolder()
-        {
-            return folder.getPath();
-        }
-
-        static boolean isLogFile(String name)
-        {
-            return TransactionFile.FILE_REGEX.matcher(name).matches();
-        }
-
-        @VisibleForTesting
-        TransactionFile getLogFile()
-        {
-            return file;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("[%s]", file.toString());
-        }
-    }
-
-    private final Tracker tracker;
-    private final TransactionData data;
-    private final Ref<TransactionLog> selfRef;
-    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
-    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
-    // Additionally, we need to make sure to delete the data file first, so on restart the others
-    // will be recognized as GCable.
-    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
-
-    TransactionLog(OperationType opType, CFMetaData metadata)
-    {
-        this(opType, metadata, null);
-    }
-
-    TransactionLog(OperationType opType, CFMetaData metadata, Tracker tracker)
-    {
-        this(opType, new Directories(metadata), tracker);
-    }
-
-    TransactionLog(OperationType opType, Directories directories, Tracker tracker)
-    {
-        this(opType, directories.getDirectoryForNewSSTables(), tracker);
-    }
-
-    TransactionLog(OperationType opType, File folder, Tracker tracker)
-    {
-        this.tracker = tracker;
-        this.data = new TransactionData(opType,
-                                        folder,
-                                        UUIDGen.getTimeUUID());
-        this.selfRef = new Ref<>(this, new TransactionTidier(data));
-
-        if (logger.isDebugEnabled())
-            logger.debug("Created transaction logs with id {}", data.id);
-    }
-
-    /**
-     * Track a reader as new.
-     **/
-    void trackNew(SSTable table)
-    {
-        if (!data.file.add(RecordType.ADD, table))
-            throw new IllegalStateException(table + " is already tracked as new");
-    }
-
-    /**
-     * Stop tracking a reader as new.
-     */
-    void untrackNew(SSTable table)
-    {
-        data.file.remove(RecordType.ADD, table);
-    }
-
-    /**
-     * Schedule a reader for deletion as soon as it is fully unreferenced and the transaction
-     * has been committed.
-     */
-    SSTableTidier obsoleted(SSTableReader reader)
-    {
-        if (data.file.contains(RecordType.ADD, reader))
-        {
-            if (data.file.contains(RecordType.REMOVE, reader))
-                throw new IllegalArgumentException();
-
-            return new SSTableTidier(reader, true, this);
-        }
-
-        if (!data.file.add(RecordType.REMOVE, reader))
-            throw new IllegalStateException();
-
-        if (tracker != null)
-            tracker.notifyDeleting(reader);
-
-        return new SSTableTidier(reader, false, this);
-    }
-
-    OperationType getType()
-    {
-        return data.getType();
-    }
-
-    UUID getId()
-    {
-        return data.getId();
-    }
-
-    @VisibleForTesting
-    String getDataFolder()
-    {
-        return data.getFolder();
-    }
-
-    @VisibleForTesting
-    TransactionData getData()
-    {
-        return data;
-    }
-
-    private static void delete(File file)
-    {
-        try
-        {
-            if (logger.isDebugEnabled())
-                logger.debug("Deleting {}", file);
-
-            Files.delete(file.toPath());
-        }
-        catch (NoSuchFileException e)
-        {
-            logger.error("Unable to delete {} as it does not exist", file);
-        }
-        catch (IOException e)
-        {
-            logger.error("Unable to delete {}", file, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * The transaction tidier.
-     *
-     * When the transaction reference is fully released we try to delete all the obsolete files
-     * depending on the transaction result, as well as the transaction log file.
-     */
-    private static class TransactionTidier implements RefCounted.Tidy, Runnable
-    {
-        private final TransactionData data;
-
-        public TransactionTidier(TransactionData data)
-        {
-            this.data = data;
-        }
-
-        public void tidy() throws Exception
-        {
-            run();
-        }
-
-        public String name()
-        {
-            return data.toString();
-        }
-
-        public void run()
-        {
-            if (logger.isDebugEnabled())
-                logger.debug("Removing files for transaction {}", name());
-
-            assert data.completed() : "Expected a completed transaction: " + data;
-
-            Throwable err = data.removeUnfinishedLeftovers(null);
-
-            if (err != null)
-            {
-                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
-                failedDeletions.add(this);
-            }
-            else
-            {
-                if (logger.isDebugEnabled())
-                    logger.debug("Closing file transaction {}", name());
-                data.close();
-            }
-        }
-    }
-
-    static class Obsoletion
-    {
-        final SSTableReader reader;
-        final SSTableTidier tidier;
-
-        public Obsoletion(SSTableReader reader, SSTableTidier tidier)
-        {
-            this.reader = reader;
-            this.tidier = tidier;
-        }
-    }
-
-    /**
-     * The SSTableReader tidier. When a reader is fully released and no longer referenced
-     * by any one, we run this. It keeps a reference to the parent transaction and releases
-     * it when done, so that the final transaction cleanup can run when all obsolete readers
-     * are released.
-     */
-    public static class SSTableTidier implements Runnable
-    {
-        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
-        private final Descriptor desc;
-        private final long sizeOnDisk;
-        private final Tracker tracker;
-        private final boolean wasNew;
-        private final Ref<TransactionLog> parentRef;
-
-        public SSTableTidier(SSTableReader referent, boolean wasNew, TransactionLog parent)
-        {
-            this.desc = referent.descriptor;
-            this.sizeOnDisk = referent.bytesOnDisk();
-            this.tracker = parent.tracker;
-            this.wasNew = wasNew;
-            this.parentRef = parent.selfRef.tryRef();
-        }
-
-        public void run()
-        {
-            SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
-
-            try
-            {
-                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
-                File datafile = new File(desc.filenameFor(Component.DATA));
-
-                delete(datafile);
-                // let the remainder be cleaned up by delete
-                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
-            }
-            catch (Throwable t)
-            {
-                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
-                failedDeletions.add(this);
-                return;
-            }
-
-            if (tracker != null && tracker.cfstore != null && !wasNew)
-                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
-
-            // release the referent to the parent so that the all transaction files can be released
-            parentRef.release();
-        }
-
-        public void abort()
-        {
-            parentRef.release();
-        }
-    }
-
-    /**
-     * Retry all deletions that failed the first time around (presumably b/c the sstable was still mmap'd.)
-     * Useful because there are times when we know GC has been invoked; also exposed as an mbean.
-     */
-    public static void rescheduleFailedDeletions()
-    {
-        Runnable task;
-        while ( null != (task = failedDeletions.poll()))
-            ScheduledExecutors.nonPeriodicTasks.submit(task);
-
-        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
-        SnapshotDeletingTask.rescheduleFailedTasks();
-    }
-
-    /**
-     * Deletions run on the nonPeriodicTasks executor, (both failedDeletions or global tidiers in SSTableReader)
-     * so by scheduling a new empty task and waiting for it we ensure any prior deletion has completed.
-     */
-    public static void waitForDeletions()
-    {
-        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
-    }
-
-    @VisibleForTesting
-    Throwable complete(Throwable accumulate)
-    {
-        try
-        {
-            accumulate = selfRef.ensureReleased(accumulate);
-            return accumulate;
-        }
-        catch (Throwable t)
-        {
-            logger.error("Failed to complete file transaction {}", getId(), t);
-            return Throwables.merge(accumulate, t);
-        }
-    }
-
-    protected Throwable doCommit(Throwable accumulate)
-    {
-        data.file.commit();
-        return complete(accumulate);
-    }
-
-    protected Throwable doAbort(Throwable accumulate)
-    {
-        data.file.abort();
-        return complete(accumulate);
-    }
-
-    protected void doPrepare() { }
-
-    /**
-     * Called on startup to scan existing folders for any unfinished leftovers of
-     * operations that were ongoing when the process exited. Also called by the standalone
-     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
-     *
-     */
-    static void removeUnfinishedLeftovers(CFMetaData metadata)
-    {
-        Throwable accumulate = null;
-
-        for (File dir : new Directories(metadata).getCFDirectories())
-        {
-            File[] logs = dir.listFiles((dir1, name) -> TransactionData.isLogFile(name));
-
-            for (File log : logs)
-            {
-                try (TransactionData data = TransactionData.make(log))
-                {
-                    accumulate = data.readLogFile(accumulate);
-                    if (accumulate == null)
-                        accumulate = data.removeUnfinishedLeftovers(accumulate);
-                    else
-                        logger.error("Unexpected disk state: failed to read transaction log {}", log, accumulate);
-                }
-            }
-        }
-
-        if (accumulate != null)
-            logger.error("Failed to remove unfinished transaction leftovers", accumulate);
-    }
-
-    /**
-     * A class for listing files in a folder. If we fail we try a few more times
-     * in case we are reading txn log files that are still being mutated.
-     */
-    static final class FileLister
-    {
-        // The maximum number of attempts for scanning the folder
-        private static final int MAX_ATTEMPTS = 10;
-
-        // The delay between each attempt
-        private static final int REATTEMPT_DELAY_MILLIS = 5;
-
-        // The folder to scan
-        private final Path folder;
-
-        // The filter determines which files the client wants returned, we pass to the filter
-        // the file and its type
-        private final BiFunction<File, FileType, Boolean> filter;
-
-        // This determines the behavior when we fail to read a txn log file after a few times (MAX_ATTEMPTS)
-        private final OnTxnErr onTxnErr;
-
-        // Each time we scan the folder we increment this counter, we scan at most for MAX_ATTEMPTS
-        private int attempts;
-
-        public FileLister(Path folder, BiFunction<File, FileType, Boolean> filter, OnTxnErr onTxnErr)
-        {
-            this.folder = folder;
-            this.filter = filter;
-            this.onTxnErr = onTxnErr;
-            this.attempts = 0;
-        }
-
-        public List<File> list()
-        {
-            while(true)
-            {
-                try
-                {
-                    return attemptList();
-                }
-                catch (Throwable t)
-                {
-                    if (attempts >= MAX_ATTEMPTS)
-                        throw new RuntimeException(String.format("Failed to list files in %s after multiple attempts, giving up", folder), t);
-
-                    logger.warn("Failed to list files in {} : {}", folder, t.getMessage());
-                    try
-                    {
-                        Thread.sleep(REATTEMPT_DELAY_MILLIS);
-                    }
-                    catch (InterruptedException e)
-                    {
-                        logger.error("Interrupted whilst waiting to reattempt listing files in {}, giving up", folder, e);
-                        throw new RuntimeException(String.format("Failed to list files in %s due to interruption, giving up", folder), t);
-                    }
-                }
-            }
-        }
-
-        List<File> attemptList() throws IOException
-        {
-            attempts++;
-
-            Map<File, FileType> files = new HashMap<>();
-            try (DirectoryStream<Path> in = Files.newDirectoryStream(folder))
-            {
-                if (!(in instanceof SecureDirectoryStream))
-                    noSpamLogger.warn("This platform does not support atomic directory streams (SecureDirectoryStream); " +
-                                       "race conditions when loading sstable files could occurr");
-
-                in.forEach(path ->
-                           {
-                               File file = path.toFile();
-                               if (file.isDirectory())
-                                   return;
-
-                               if (TransactionData.isLogFile(file.getName()))
-                               {
-                                   Set<File> tmpFiles = getTemporaryFiles(file);
-                                   if (tmpFiles != null)
-                                   { // process the txn log file only if we can read it (tmpFiles != null)
-                                       tmpFiles.stream().forEach((f) -> files.put(f, FileType.TEMPORARY));
-                                       files.put(file, FileType.TXN_LOG);
-                                   }
-                               }
-                               else
-                               {
-                                   files.putIfAbsent(file, FileType.FINAL);
-                               }
-                           });
-            }
-
-            return files.entrySet().stream()
-                        .filter((e) -> filter.apply(e.getKey(), e.getValue()))
-                        .map(Map.Entry::getKey)
-                        .collect(Collectors.toList());
-        }
-
-        Set<File> getTemporaryFiles(File file)
-        {
-            try (TransactionData txn = TransactionData.make(file))
-            {
-                maybeFail(txn.readLogFile(null));
-                return txn.getTemporaryFiles();
-            }
-            catch(Throwable t)
-            {
-                // We always fail if the onTxnErr is set to THROW or if we haven't
-                // reached the maximum number of attempts yet. Otherwise
-                // we just log an error and continue as if the txn log file does not exist
-                // clients can choose which behavior they want via onTxnLogError
-                if (attempts < MAX_ATTEMPTS ||
-                    onTxnErr == OnTxnErr.THROW)
-                    throw new RuntimeException(t);
-
-                logger.error("Failed to read temporary files of txn log {}", file, t);
-                return null; // txn.getTemporaryFiles() could be empty so we must use null to differentiate
-            }
-        }
-    }
-
-    @VisibleForTesting
-    static Set<File> getTemporaryFiles(CFMetaData metadata, File folder)
-    {
-        Set<File> ret = new HashSet<>();
-
-        List<File> directories = new Directories(metadata).getCFDirectories();
-        directories.add(folder);
-        for (File dir : directories)
-            ret.addAll(new FileLister(dir.toPath(),
-                                      (file, type) -> type != FileType.FINAL,
-                                      OnTxnErr.IGNORE).list());
-
-        return ret;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 63b8f3e..811f4ef 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -161,7 +161,6 @@ public abstract class SSTable
         return descriptor.ksname;
     }
 
-    @VisibleForTesting
     public List<String> getAllFilePaths()
     {
         List<String> ret = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index b958240..d8ff36a 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -46,7 +46,6 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.db.rows.SliceableUnfilteredRowIterator;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
@@ -1646,7 +1645,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
      * @return true if the this is the first time the file was marked obsolete.  Calling this
      * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
      */
-    public void markObsolete(TransactionLog.SSTableTidier tidier)
+    public void markObsolete(Runnable tidier)
     {
         if (logger.isDebugEnabled())
             logger.debug("Marking {} compacted", getFilename());
@@ -2182,7 +2181,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         // sstable have been released
         private ScheduledFuture readMeterSyncFuture;
         // shared state managing if the logical sstable has been compacted; this is used in cleanup
-        private volatile TransactionLog.SSTableTidier obsoletion;
+        private volatile Runnable obsoletion;
 
         GlobalTidy(final SSTableReader reader)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 920eee0..8b7b1e1 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -25,6 +25,7 @@ import java.nio.charset.StandardCharsets;
 import java.nio.file.*;
 import java.text.DecimalFormat;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import sun.nio.ch.DirectBuffer;
@@ -320,16 +321,16 @@ public class FileUtils
     }
 
     /** Convert absolute path into a path relative to the base path */
-    public static String getRelativePath(String basePath, String absolutePath)
+    public static String getRelativePath(String basePath, String path)
     {
         try
         {
-            return Paths.get(basePath).relativize(Paths.get(absolutePath)).toString();
+            return Paths.get(basePath).relativize(Paths.get(path)).toString();
         }
         catch(Exception ex)
         {
             String absDataPath = FileUtils.getCanonicalPath(basePath);
-            return Paths.get(absDataPath).relativize(Paths.get(absolutePath)).toString();
+            return Paths.get(absDataPath).relativize(Paths.get(path)).toString();
         }
     }
 
@@ -619,6 +620,9 @@ public class FileUtils
         }
         catch (IOException ex)
         {
+            if (ex instanceof NoSuchFileException)
+                return Collections.emptyList();
+
             throw new RuntimeException(ex);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/service/GCInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java
index 252d1c3..21ecdb0 100644
--- a/src/java/org/apache/cassandra/service/GCInspector.java
+++ b/src/java/org/apache/cassandra/service/GCInspector.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
 import com.sun.management.GarbageCollectionNotificationInfo;
 import com.sun.management.GcInfo;
 
-import org.apache.cassandra.db.lifecycle.TransactionLog;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.utils.StatusLogger;
 
 public class GCInspector implements NotificationListener, GCInspectorMXBean
@@ -284,7 +284,7 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
 
             // if we just finished an old gen collection and we're still using a lot of memory, try to reduce the pressure
             if (gcState.assumeGCIsOldGen)
-                TransactionLog.rescheduleFailedDeletions();
+                LifecycleTransaction.rescheduleFailedDeletions();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 2d9bbec..fc2153f 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -60,7 +60,7 @@ import org.apache.cassandra.batchlog.BatchRemoveVerbHandler;
 import org.apache.cassandra.batchlog.BatchlogManager;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.BootStrapper;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
@@ -4262,7 +4262,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public void rescheduleFailedDeletions()
     {
-        TransactionLog.rescheduleFailedDeletions();
+        LifecycleTransaction.rescheduleFailedDeletions();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index f3a1a35..f82cd39 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -34,7 +34,6 @@ import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.*;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -145,7 +144,7 @@ public class StandaloneScrubber
             // Check (and repair) manifests
             checkManifest(cfs.getCompactionStrategyManager(), cfs, sstables);
             CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             System.exit(0); // We need that to stop non daemonized threads
         }
         catch (Exception e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
index a192491..3c4f7dc 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -23,7 +23,6 @@ import java.util.*;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
@@ -164,7 +163,7 @@ public class StandaloneSplitter
                 }
             }
             CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             System.exit(0); // We need that to stop non daemonized threads
         }
         catch (Exception e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index cf94c99..a868917 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.tools;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
@@ -120,7 +119,7 @@ public class StandaloneUpgrader
                 }
             }
             CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             System.exit(0);
         }
         catch (Exception e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/utils/CLibrary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CLibrary.java b/src/java/org/apache/cassandra/utils/CLibrary.java
index 645a4ce..f5da13a 100644
--- a/src/java/org/apache/cassandra/utils/CLibrary.java
+++ b/src/java/org/apache/cassandra/utils/CLibrary.java
@@ -264,7 +264,7 @@ public final class CLibrary
             if (!(e instanceof LastErrorException))
                 throw e;
 
-            logger.warn(String.format("fsync(%d) failed, errno (%d).", fd, errno(e)));
+            logger.warn(String.format("fsync(%d) failed, errno (%d) {}", fd, errno(e)), e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index aedba0e..569451b 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -32,12 +32,11 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.KeyCacheKey;
-import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -164,7 +163,7 @@ public class KeyCacheTest
 
         refs.release();
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         // after releasing the reference this should drop to 2
         assertKeyCacheSize(2, KEYSPACE1, COLUMN_FAMILY1);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index e9bf4c5..2fc8436 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -37,7 +37,6 @@ import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.compaction.Scrubber;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.db.partitions.Partition;
@@ -370,7 +369,7 @@ public class ScrubTest
             {
                 scrubber.scrub();
             }
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             cfs.loadNewSSTables();
             assertOrderedAll(cfs, 7);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
index 7f1b2bd..0488245 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
@@ -159,10 +159,10 @@ public class HelpersTest
     public void testMarkObsolete()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        TransactionLog txnLogs = new TransactionLog(OperationType.UNKNOWN, cfs.metadata);
+        LogTransaction txnLogs = new LogTransaction(OperationType.UNKNOWN, cfs.metadata);
         Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
 
-        List<TransactionLog.Obsoletion> obsoletions = new ArrayList<>();
+        List<LogTransaction.Obsoletion> obsoletions = new ArrayList<>();
         Assert.assertNull(Helpers.prepareForObsoletion(readers, txnLogs, obsoletions, null));
         assertNotNull(obsoletions);
         assertEquals(2, obsoletions.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index db27662..6435e3e 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -249,7 +249,7 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
 
     protected TestableTransaction newTest()
     {
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
         SSTableReader.resetTidying();
         return new TxnTest();
     }


[2/7] cassandra git commit: Handle non-atomic directory streams safely (CASSANDRA-10109)

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java b/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
deleted file mode 100644
index 69de370..0000000
--- a/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
+++ /dev/null
@@ -1,1141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.lifecycle;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.NoSuchFileException;
-import java.nio.file.Path;
-import java.nio.file.SecureDirectoryStream;
-import java.util.*;
-import java.util.Objects;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.zip.CRC32;
-import java.util.zip.Checksum;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.Runnables;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.compaction.OperationType;
-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.SnapshotDeletingTask;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.format.big.BigFormat;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.concurrent.Ref;
-import org.apache.cassandra.utils.concurrent.RefCounted;
-import org.apache.cassandra.utils.concurrent.Transactional;
-
-import static org.apache.cassandra.db.Directories.OnTxnErr;
-import static org.apache.cassandra.db.Directories.FileType;
-import static org.apache.cassandra.utils.Throwables.maybeFail;
-import static org.apache.cassandra.utils.Throwables.merge;
-
-/**
- * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
- * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
- * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
- * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
- *
- * A class that tracks sstable files involved in a transaction across sstables:
- * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
- *
- * The transaction log file contains new and old sstables as follows:
- *
- * add:[sstable-2][CRC]
- * remove:[sstable-1,max_update_time,num files][CRC]
- *
- * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
- * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
- * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
- * and file sizes.
- *
- * Upon commit we add a final line to the log file:
- *
- * commit:[commit_time][CRC]
- *
- * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
- * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
- * was committed, vice-versa if the transaction was aborted.
- *
- * On start-up we look for any transaction log files and repeat the cleanup process described above.
- *
- * See CASSANDRA-7066 for full details.
- */
-public class TransactionLog extends Transactional.AbstractTransactional implements Transactional
-{
-    private static final Logger logger = LoggerFactory.getLogger(TransactionLog.class);
-    private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1L, TimeUnit.HOURS);
-
-    /**
-     * If the format of the lines in the transaction log is wrong or the checksum
-     * does not match, then we throw this exception.
-     */
-    public static final class CorruptTransactionLogException extends RuntimeException
-    {
-        public final TransactionFile file;
-
-        public CorruptTransactionLogException(String message, TransactionFile file)
-        {
-            super(message);
-            this.file = file;
-        }
-    }
-
-    public enum RecordType
-    {
-        ADD,    // new files to be retained on commit
-        REMOVE, // old files to be retained on abort
-        COMMIT, // commit flag
-        ABORT;  // abort flag
-        public static RecordType fromPrefix(String prefix)
-        {
-            return valueOf(prefix.toUpperCase());
-        }
-    }
-
-    /**
-     * A log file record, each record is encoded in one line and has different
-     * content depending on the record type.
-     */
-    final static class Record
-    {
-        public final RecordType type;
-        public final String relativeFilePath;
-        public final long updateTime;
-        public final int numFiles;
-        public final String record;
-
-        static String REGEX_STR = "^(add|remove|commit|abort):\\[([^,]*),?([^,]*),?([^,]*)\\]$";
-        static Pattern REGEX = Pattern.compile(REGEX_STR, Pattern.CASE_INSENSITIVE); // (add|remove|commit|abort):[*,*,*]
-
-        public static Record make(String record, boolean isLast)
-        {
-            try
-            {
-                Matcher matcher = REGEX.matcher(record);
-                if (!matcher.matches() || matcher.groupCount() != 4)
-                    throw new IllegalStateException(String.format("Invalid record \"%s\"", record));
-
-                RecordType type = RecordType.fromPrefix(matcher.group(1));
-                return new Record(type, matcher.group(2), Long.valueOf(matcher.group(3)), Integer.valueOf(matcher.group(4)), record);
-            }
-            catch (Throwable t)
-            {
-                if (!isLast)
-                    throw t;
-
-                int pos = record.indexOf(':');
-                if (pos <= 0)
-                    throw t;
-
-                RecordType recordType;
-                try
-                {
-                    recordType = RecordType.fromPrefix(record.substring(0, pos));
-                }
-                catch (Throwable ignore)
-                {
-                    throw t;
-                }
-
-                return new Record(recordType, "", 0, 0, record);
-
-            }
-        }
-
-        public static Record makeCommit(long updateTime)
-        {
-            return new Record(RecordType.COMMIT, "", updateTime, 0, "");
-        }
-
-        public static Record makeAbort(long updateTime)
-        {
-            return new Record(RecordType.ABORT, "", updateTime, 0, "");
-        }
-
-        public static Record makeNew(String relativeFilePath)
-        {
-            return new Record(RecordType.ADD, relativeFilePath, 0, 0, "");
-        }
-
-        public static Record makeOld(String parentFolder, String relativeFilePath)
-        {
-            return makeOld(getTrackedFiles(parentFolder, relativeFilePath), relativeFilePath);
-        }
-
-        public static Record makeOld(List<File> files, String relativeFilePath)
-        {
-            long lastModified = files.stream()
-                                     .mapToLong(File::lastModified)
-                                     .reduce(0L, Long::max);
-            return new Record(RecordType.REMOVE, relativeFilePath, lastModified, files.size(), "");
-        }
-
-        private Record(RecordType type,
-                       String relativeFilePath,
-                       long updateTime,
-                       int numFiles,
-                       String record)
-        {
-            this.type = type;
-            this.relativeFilePath = hasFilePath(type) ? relativeFilePath : ""; // only meaningful for some records
-            this.updateTime = type == RecordType.REMOVE ? updateTime : 0; // only meaningful for old records
-            this.numFiles = type == RecordType.REMOVE ? numFiles : 0; // only meaningful for old records
-            this.record = record.isEmpty() ? format() : record;
-        }
-
-        private static boolean hasFilePath(RecordType type)
-        {
-            return type == RecordType.ADD || type == RecordType.REMOVE;
-        }
-
-        private String format()
-        {
-            return String.format("%s:[%s,%d,%d]", type.toString(), relativeFilePath, updateTime, numFiles);
-        }
-
-        public byte[] getBytes()
-        {
-            return record.getBytes(FileUtils.CHARSET);
-        }
-
-        public boolean verify(String parentFolder, boolean lastRecordIsCorrupt)
-        {
-            if (type != RecordType.REMOVE)
-                return true;
-
-            List<File> files = getTrackedFiles(parentFolder);
-
-            // Paranoid sanity checks: we create another record by looking at the files as they are
-            // on disk right now and make sure the information still matches
-            Record currentRecord = Record.makeOld(files, relativeFilePath);
-            if (updateTime != currentRecord.updateTime && currentRecord.numFiles > 0)
-            {
-                logger.error("Unexpected files detected for sstable [{}], record [{}]: last update time [{}] should have been [{}]",
-                             relativeFilePath,
-                             record,
-                             new Date(currentRecord.updateTime),
-                             new Date(updateTime));
-                return false;
-            }
-
-            if (lastRecordIsCorrupt && currentRecord.numFiles < numFiles)
-            { // if we found a corruption in the last record, then we continue only if the number of files matches exactly.
-                logger.error("Unexpected files detected for sstable [{}], record [{}]: number of files [{}] should have been [{}]",
-                             relativeFilePath,
-                             record,
-                             currentRecord.numFiles,
-                             numFiles);
-                return false;
-            }
-
-            return true;
-        }
-
-        public List<File> getTrackedFiles(String parentFolder)
-        {
-            if (!hasFilePath(type))
-                return Collections.emptyList();
-
-            return getTrackedFiles(parentFolder, relativeFilePath);
-        }
-
-        public static List<File> getTrackedFiles(String parentFolder, String relativeFilePath)
-        {
-            return Arrays.asList(new File(parentFolder).listFiles((dir, name) -> name.startsWith(relativeFilePath)));
-        }
-
-        @Override
-        public int hashCode()
-        {
-            // see comment in equals
-            return Objects.hash(type, relativeFilePath);
-        }
-
-        @Override
-        public boolean equals(Object obj)
-        {
-            if (obj == null)
-                return false;
-
-            if (getClass() != obj.getClass())
-                return false;
-
-            final Record other = (Record)obj;
-
-            // we exclude on purpose checksum, update time and count as
-            // we don't want duplicated records that differ only by
-            // properties that might change on disk, especially COMMIT records,
-            // there should be only one regardless of update time
-            return type.equals(other.type) &&
-                   relativeFilePath.equals(other.relativeFilePath);
-        }
-
-        @Override
-        public String toString()
-        {
-            return record;
-        }
-    }
-
-    /**
-     * The transaction log file, which contains many records.
-     */
-    final static class TransactionFile
-    {
-        static String EXT = ".log";
-        static char SEP = '_';
-        // cc_txn_opname_id.log (where cc is one of the sstable versions defined in BigVersion)
-        static String FILE_REGEX_STR = String.format("^(.{2})_txn_(.*)_(.*)%s$", EXT);
-        static Pattern FILE_REGEX = Pattern.compile(FILE_REGEX_STR);
-        static String LINE_REGEX_STR = "^(.*)\\[(\\d*)\\]$"; // *[checksum]
-        static Pattern LINE_REGEX = Pattern.compile(LINE_REGEX_STR);
-
-        public final File file;
-        public final TransactionData parent;
-        public final Set<Record> records = new HashSet<>();
-        public final Checksum checksum = new CRC32();
-
-        public TransactionFile(TransactionData parent)
-        {
-            this.file = new File(parent.getFileName());
-            this.parent = parent;
-        }
-
-        public void readRecords()
-        {
-            records.clear();
-            checksum.reset();
-
-            Iterator<String> it = FileUtils.readLines(file).iterator();
-            while(it.hasNext())
-                records.add(readRecord(it.next(), !it.hasNext())); // JLS execution order is left-to-right
-
-            for (Record record : records)
-            {
-                if (!record.verify(parent.getFolder(), false))
-                    throw new CorruptTransactionLogException(String.format("Failed to verify transaction %s record [%s]: unexpected disk state, aborting", parent.getId(), record),
-                                                             this);
-            }
-        }
-
-        private Record readRecord(String line, boolean isLast)
-        {
-            Matcher matcher = LINE_REGEX.matcher(line);
-            if (!matcher.matches() || matcher.groupCount() != 2)
-            {
-                handleReadRecordError(String.format("cannot parse line \"%s\"", line), isLast);
-                return Record.make(line, isLast);
-            }
-
-            byte[] bytes = matcher.group(1).getBytes(FileUtils.CHARSET);
-            checksum.update(bytes, 0, bytes.length);
-
-            if (checksum.getValue() != Long.valueOf(matcher.group(2)))
-                handleReadRecordError(String.format("invalid line checksum %s for \"%s\"", matcher.group(2), line), isLast);
-
-            try
-            {
-                return Record.make(matcher.group(1), isLast);
-            }
-            catch (Throwable t)
-            {
-                throw new CorruptTransactionLogException(String.format("Cannot make record \"%s\": %s", line, t.getMessage()), this);
-            }
-        }
-
-        private void handleReadRecordError(String message, boolean isLast)
-        {
-            if (isLast)
-            {
-                for (Record record : records)
-                {
-                    if (!record.verify(parent.getFolder(), true))
-                        throw new CorruptTransactionLogException(String.format("Last record of transaction %s is corrupt [%s] and at least " +
-                                                                               "one previous record does not match state on disk, unexpected disk state, aborting",
-                                                                               parent.getId(), message),
-                                                                 this);
-                }
-
-                // if only the last record is corrupt and all other records have matching files on disk, @see verifyRecord,
-                // then we simply exited whilst serializing the last record and we carry on
-                logger.warn(String.format("Last record of transaction %s is corrupt or incomplete [%s], but all previous records match state on disk; continuing", parent.getId(), message));
-
-            }
-            else
-            {
-                throw new CorruptTransactionLogException(String.format("Non-last record of transaction %s is corrupt [%s], unexpected disk state, aborting", parent.getId(), message), this);
-            }
-        }
-
-        public void commit()
-        {
-            assert !completed() : "Already completed!";
-            addRecord(Record.makeCommit(System.currentTimeMillis()));
-        }
-
-        public void abort()
-        {
-            assert !completed() : "Already completed!";
-            addRecord(Record.makeAbort(System.currentTimeMillis()));
-        }
-
-        public boolean committed()
-        {
-            return records.contains(Record.makeCommit(0));
-        }
-
-        public boolean aborted()
-        {
-            return records.contains(Record.makeAbort(0));
-        }
-
-        public boolean completed()
-        {
-            return committed() || aborted();
-        }
-
-        public boolean add(RecordType type, SSTable table)
-        {
-            Record record = makeRecord(type, table);
-            if (records.contains(record))
-                return false;
-
-            addRecord(record);
-            return true;
-        }
-
-        private Record makeRecord(RecordType type, SSTable table)
-        {
-            String relativePath = FileUtils.getRelativePath(parent.getFolder(), table.descriptor.baseFilename());
-            if (type == RecordType.ADD)
-            {
-                return Record.makeNew(relativePath);
-            }
-            else if (type == RecordType.REMOVE)
-            {
-                return Record.makeOld(parent.getFolder(), relativePath);
-            }
-            else
-            {
-                throw new AssertionError("Invalid record type " + type);
-            }
-        }
-
-        private void addRecord(Record record)
-        {
-            // we only checksum the records, not the checksums themselves
-            byte[] bytes = record.getBytes();
-            checksum.update(bytes, 0, bytes.length);
-
-            records.add(record);
-            FileUtils.append(file, String.format("%s[%d]", record, checksum.getValue()));
-
-            parent.sync();
-        }
-
-        public void remove(RecordType type, SSTable table)
-        {
-            Record record = makeRecord(type, table);
-
-            assert records.contains(record) : String.format("[%s] is not tracked by %s", record, file);
-
-            records.remove(record);
-            deleteRecord(record);
-        }
-
-        public boolean contains(RecordType type, SSTable table)
-        {
-            return records.contains(makeRecord(type, table));
-        }
-
-        public void deleteRecords(RecordType type)
-        {
-            assert file.exists() : String.format("Expected %s to exists", file);
-            records.stream()
-                   .filter((r) -> r.type == type)
-                   .forEach(this::deleteRecord);
-            records.clear();
-        }
-
-        private void deleteRecord(Record record)
-        {
-            List<File> files = record.getTrackedFiles(parent.getFolder());
-            if (files.isEmpty())
-                return; // Files no longer exist, nothing to do
-
-            // we sort the files in ascending update time order so that the last update time
-            // stays the same even if we only partially delete files
-            files.sort((f1, f2) -> Long.compare(f1.lastModified(), f2.lastModified()));
-
-            files.forEach(TransactionLog::delete);
-        }
-
-        public Set<File> getTrackedFiles(RecordType type)
-        {
-            return records.stream()
-                          .filter((r) -> r.type == type)
-                          .map((r) -> r.getTrackedFiles(parent.getFolder()))
-                          .flatMap(List::stream)
-                          .collect(Collectors.toSet());
-        }
-
-        public void delete()
-        {
-            TransactionLog.delete(file);
-        }
-
-        public boolean exists()
-        {
-            return file.exists();
-        }
-
-        @Override
-        public String toString()
-        {
-            return FileUtils.getRelativePath(parent.getFolder(), FileUtils.getCanonicalPath(file));
-        }
-    }
-
-    /**
-     * We split the transaction data from TransactionLog that implements the behavior
-     * because we need to reconstruct any left-overs and clean them up, as well as work
-     * out which files are temporary. So for these cases we don't want the full
-     * transactional behavior, plus it's handy for the TransactionTidier.
-     */
-    final static class TransactionData implements AutoCloseable
-    {
-        private final OperationType opType;
-        private final UUID id;
-        private final File folder;
-        private final TransactionFile file;
-        private int folderDescriptor;
-
-        static TransactionData make(File logFile)
-        {
-            Matcher matcher = TransactionFile.FILE_REGEX.matcher(logFile.getName());
-            assert matcher.matches() && matcher.groupCount() == 3;
-
-            // For now we don't need this but it is there in case we need to change
-            // file format later on, the version is the sstable version as defined in BigFormat
-            //String version = matcher.group(1);
-
-            OperationType operationType = OperationType.fromFileName(matcher.group(2));
-            UUID id = UUID.fromString(matcher.group(3));
-
-            return new TransactionData(operationType, logFile.getParentFile(), id);
-        }
-
-        TransactionData(OperationType opType, File folder, UUID id)
-        {
-            this.opType = opType;
-            this.id = id;
-            this.folder = folder;
-            this.file = new TransactionFile(this);
-            this.folderDescriptor = CLibrary.tryOpenDirectory(folder.getPath());
-        }
-
-        public Throwable readLogFile(Throwable accumulate)
-        {
-            try
-            {
-                file.readRecords();
-            }
-            catch (Throwable t)
-            {
-                accumulate = merge(accumulate, t);
-            }
-
-            return accumulate;
-        }
-
-        public void close()
-        {
-            if (folderDescriptor > 0)
-            {
-                CLibrary.tryCloseFD(folderDescriptor);
-                folderDescriptor = -1;
-            }
-        }
-
-        void sync()
-        {
-            if (folderDescriptor > 0)
-                CLibrary.trySync(folderDescriptor);
-        }
-
-        OperationType getType()
-        {
-            return opType;
-        }
-
-        UUID getId()
-        {
-            return id;
-        }
-
-        boolean completed()
-        {
-            return  file.completed();
-        }
-
-        Throwable removeUnfinishedLeftovers(Throwable accumulate)
-        {
-            try
-            {
-                if (file.committed())
-                    file.deleteRecords(RecordType.REMOVE);
-                else
-                    file.deleteRecords(RecordType.ADD);
-
-                // we sync the parent file descriptor between contents and log deletion
-                // to ensure there is a happens before edge between them
-                sync();
-
-                file.delete();
-            }
-            catch (Throwable t)
-            {
-                accumulate = merge(accumulate, t);
-            }
-
-            return accumulate;
-        }
-
-        Set<File> getTemporaryFiles()
-        {
-            sync();
-
-            if (!file.exists())
-                return Collections.emptySet();
-
-            if (file.committed())
-                return file.getTrackedFiles(RecordType.REMOVE);
-            else
-                return file.getTrackedFiles(RecordType.ADD);
-        }
-
-        String getFileName()
-        {
-            String fileName = StringUtils.join(BigFormat.latestVersion,
-                                               TransactionFile.SEP,
-                                               "txn",
-                                               TransactionFile.SEP,
-                                               opType.fileName,
-                                               TransactionFile.SEP,
-                                               id.toString(),
-                                               TransactionFile.EXT);
-            return StringUtils.join(folder, File.separator, fileName);
-        }
-
-        String getFolder()
-        {
-            return folder.getPath();
-        }
-
-        static boolean isLogFile(String name)
-        {
-            return TransactionFile.FILE_REGEX.matcher(name).matches();
-        }
-
-        @VisibleForTesting
-        TransactionFile getLogFile()
-        {
-            return file;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("[%s]", file.toString());
-        }
-    }
-
-    private final Tracker tracker;
-    private final TransactionData data;
-    private final Ref<TransactionLog> selfRef;
-    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
-    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
-    // Additionally, we need to make sure to delete the data file first, so on restart the others
-    // will be recognized as GCable.
-    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
-
-    TransactionLog(OperationType opType, CFMetaData metadata)
-    {
-        this(opType, metadata, null);
-    }
-
-    TransactionLog(OperationType opType, CFMetaData metadata, Tracker tracker)
-    {
-        this(opType, new Directories(metadata), tracker);
-    }
-
-    TransactionLog(OperationType opType, Directories directories, Tracker tracker)
-    {
-        this(opType, directories.getDirectoryForNewSSTables(), tracker);
-    }
-
-    TransactionLog(OperationType opType, File folder, Tracker tracker)
-    {
-        this.tracker = tracker;
-        this.data = new TransactionData(opType,
-                                        folder,
-                                        UUIDGen.getTimeUUID());
-        this.selfRef = new Ref<>(this, new TransactionTidier(data));
-
-        if (logger.isDebugEnabled())
-            logger.debug("Created transaction logs with id {}", data.id);
-    }
-
-    /**
-     * Track a reader as new.
-     **/
-    void trackNew(SSTable table)
-    {
-        if (!data.file.add(RecordType.ADD, table))
-            throw new IllegalStateException(table + " is already tracked as new");
-    }
-
-    /**
-     * Stop tracking a reader as new.
-     */
-    void untrackNew(SSTable table)
-    {
-        data.file.remove(RecordType.ADD, table);
-    }
-
-    /**
-     * Schedule a reader for deletion as soon as it is fully unreferenced and the transaction
-     * has been committed.
-     */
-    SSTableTidier obsoleted(SSTableReader reader)
-    {
-        if (data.file.contains(RecordType.ADD, reader))
-        {
-            if (data.file.contains(RecordType.REMOVE, reader))
-                throw new IllegalArgumentException();
-
-            return new SSTableTidier(reader, true, this);
-        }
-
-        if (!data.file.add(RecordType.REMOVE, reader))
-            throw new IllegalStateException();
-
-        if (tracker != null)
-            tracker.notifyDeleting(reader);
-
-        return new SSTableTidier(reader, false, this);
-    }
-
-    OperationType getType()
-    {
-        return data.getType();
-    }
-
-    UUID getId()
-    {
-        return data.getId();
-    }
-
-    @VisibleForTesting
-    String getDataFolder()
-    {
-        return data.getFolder();
-    }
-
-    @VisibleForTesting
-    TransactionData getData()
-    {
-        return data;
-    }
-
-    private static void delete(File file)
-    {
-        try
-        {
-            if (logger.isDebugEnabled())
-                logger.debug("Deleting {}", file);
-
-            Files.delete(file.toPath());
-        }
-        catch (NoSuchFileException e)
-        {
-            logger.error("Unable to delete {} as it does not exist", file);
-        }
-        catch (IOException e)
-        {
-            logger.error("Unable to delete {}", file, e);
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * The transaction tidier.
-     *
-     * When the transaction reference is fully released we try to delete all the obsolete files
-     * depending on the transaction result, as well as the transaction log file.
-     */
-    private static class TransactionTidier implements RefCounted.Tidy, Runnable
-    {
-        private final TransactionData data;
-
-        public TransactionTidier(TransactionData data)
-        {
-            this.data = data;
-        }
-
-        public void tidy() throws Exception
-        {
-            run();
-        }
-
-        public String name()
-        {
-            return data.toString();
-        }
-
-        public void run()
-        {
-            if (logger.isDebugEnabled())
-                logger.debug("Removing files for transaction {}", name());
-
-            assert data.completed() : "Expected a completed transaction: " + data;
-
-            Throwable err = data.removeUnfinishedLeftovers(null);
-
-            if (err != null)
-            {
-                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
-                failedDeletions.add(this);
-            }
-            else
-            {
-                if (logger.isDebugEnabled())
-                    logger.debug("Closing file transaction {}", name());
-                data.close();
-            }
-        }
-    }
-
-    static class Obsoletion
-    {
-        final SSTableReader reader;
-        final SSTableTidier tidier;
-
-        public Obsoletion(SSTableReader reader, SSTableTidier tidier)
-        {
-            this.reader = reader;
-            this.tidier = tidier;
-        }
-    }
-
-    /**
-     * The SSTableReader tidier. When a reader is fully released and no longer referenced
-     * by any one, we run this. It keeps a reference to the parent transaction and releases
-     * it when done, so that the final transaction cleanup can run when all obsolete readers
-     * are released.
-     */
-    public static class SSTableTidier implements Runnable
-    {
-        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
-        private final Descriptor desc;
-        private final long sizeOnDisk;
-        private final Tracker tracker;
-        private final boolean wasNew;
-        private final Ref<TransactionLog> parentRef;
-
-        public SSTableTidier(SSTableReader referent, boolean wasNew, TransactionLog parent)
-        {
-            this.desc = referent.descriptor;
-            this.sizeOnDisk = referent.bytesOnDisk();
-            this.tracker = parent.tracker;
-            this.wasNew = wasNew;
-            this.parentRef = parent.selfRef.tryRef();
-        }
-
-        public void run()
-        {
-            SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
-
-            try
-            {
-                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
-                File datafile = new File(desc.filenameFor(Component.DATA));
-
-                delete(datafile);
-                // let the remainder be cleaned up by delete
-                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
-            }
-            catch (Throwable t)
-            {
-                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
-                failedDeletions.add(this);
-                return;
-            }
-
-            if (tracker != null && tracker.cfstore != null && !wasNew)
-                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
-
-            // release the referent to the parent so that the all transaction files can be released
-            parentRef.release();
-        }
-
-        public void abort()
-        {
-            parentRef.release();
-        }
-    }
-
-    /**
-     * Retry all deletions that failed the first time around (presumably b/c the sstable was still mmap'd.)
-     * Useful because there are times when we know GC has been invoked; also exposed as an mbean.
-     */
-    public static void rescheduleFailedDeletions()
-    {
-        Runnable task;
-        while ( null != (task = failedDeletions.poll()))
-            ScheduledExecutors.nonPeriodicTasks.submit(task);
-
-        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
-        SnapshotDeletingTask.rescheduleFailedTasks();
-    }
-
-    /**
-     * Deletions run on the nonPeriodicTasks executor, (both failedDeletions or global tidiers in SSTableReader)
-     * so by scheduling a new empty task and waiting for it we ensure any prior deletion has completed.
-     */
-    public static void waitForDeletions()
-    {
-        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
-    }
-
-    @VisibleForTesting
-    Throwable complete(Throwable accumulate)
-    {
-        try
-        {
-            accumulate = selfRef.ensureReleased(accumulate);
-            return accumulate;
-        }
-        catch (Throwable t)
-        {
-            logger.error("Failed to complete file transaction {}", getId(), t);
-            return Throwables.merge(accumulate, t);
-        }
-    }
-
-    protected Throwable doCommit(Throwable accumulate)
-    {
-        data.file.commit();
-        return complete(accumulate);
-    }
-
-    protected Throwable doAbort(Throwable accumulate)
-    {
-        data.file.abort();
-        return complete(accumulate);
-    }
-
-    protected void doPrepare() { }
-
-    /**
-     * Called on startup to scan existing folders for any unfinished leftovers of
-     * operations that were ongoing when the process exited. Also called by the standalone
-     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
-     *
-     */
-    static void removeUnfinishedLeftovers(CFMetaData metadata)
-    {
-        Throwable accumulate = null;
-
-        for (File dir : new Directories(metadata).getCFDirectories())
-        {
-            File[] logs = dir.listFiles((dir1, name) -> TransactionData.isLogFile(name));
-
-            for (File log : logs)
-            {
-                try (TransactionData data = TransactionData.make(log))
-                {
-                    accumulate = data.readLogFile(accumulate);
-                    if (accumulate == null)
-                        accumulate = data.removeUnfinishedLeftovers(accumulate);
-                    else
-                        logger.error("Unexpected disk state: failed to read transaction log {}", log, accumulate);
-                }
-            }
-        }
-
-        if (accumulate != null)
-            logger.error("Failed to remove unfinished transaction leftovers", accumulate);
-    }
-
-    /**
-     * A class for listing files in a folder. If we fail we try a few more times
-     * in case we are reading txn log files that are still being mutated.
-     */
-    static final class FileLister
-    {
-        // The maximum number of attempts for scanning the folder
-        private static final int MAX_ATTEMPTS = 10;
-
-        // The delay between each attempt
-        private static final int REATTEMPT_DELAY_MILLIS = 5;
-
-        // The folder to scan
-        private final Path folder;
-
-        // The filter determines which files the client wants returned, we pass to the filter
-        // the file and its type
-        private final BiFunction<File, FileType, Boolean> filter;
-
-        // This determines the behavior when we fail to read a txn log file after a few times (MAX_ATTEMPTS)
-        private final OnTxnErr onTxnErr;
-
-        // Each time we scan the folder we increment this counter, we scan at most for MAX_ATTEMPTS
-        private int attempts;
-
-        public FileLister(Path folder, BiFunction<File, FileType, Boolean> filter, OnTxnErr onTxnErr)
-        {
-            this.folder = folder;
-            this.filter = filter;
-            this.onTxnErr = onTxnErr;
-            this.attempts = 0;
-        }
-
-        public List<File> list()
-        {
-            while(true)
-            {
-                try
-                {
-                    return attemptList();
-                }
-                catch (Throwable t)
-                {
-                    if (attempts >= MAX_ATTEMPTS)
-                        throw new RuntimeException(String.format("Failed to list files in %s after multiple attempts, giving up", folder), t);
-
-                    logger.warn("Failed to list files in {} : {}", folder, t.getMessage());
-                    try
-                    {
-                        Thread.sleep(REATTEMPT_DELAY_MILLIS);
-                    }
-                    catch (InterruptedException e)
-                    {
-                        logger.error("Interrupted whilst waiting to reattempt listing files in {}, giving up", folder, e);
-                        throw new RuntimeException(String.format("Failed to list files in %s due to interruption, giving up", folder), t);
-                    }
-                }
-            }
-        }
-
-        List<File> attemptList() throws IOException
-        {
-            attempts++;
-
-            Map<File, FileType> files = new HashMap<>();
-            try (DirectoryStream<Path> in = Files.newDirectoryStream(folder))
-            {
-                if (!(in instanceof SecureDirectoryStream))
-                    noSpamLogger.warn("This platform does not support atomic directory streams (SecureDirectoryStream); " +
-                                       "race conditions when loading sstable files could occurr");
-
-                in.forEach(path ->
-                           {
-                               File file = path.toFile();
-                               if (file.isDirectory())
-                                   return;
-
-                               if (TransactionData.isLogFile(file.getName()))
-                               {
-                                   Set<File> tmpFiles = getTemporaryFiles(file);
-                                   if (tmpFiles != null)
-                                   { // process the txn log file only if we can read it (tmpFiles != null)
-                                       tmpFiles.stream().forEach((f) -> files.put(f, FileType.TEMPORARY));
-                                       files.put(file, FileType.TXN_LOG);
-                                   }
-                               }
-                               else
-                               {
-                                   files.putIfAbsent(file, FileType.FINAL);
-                               }
-                           });
-            }
-
-            return files.entrySet().stream()
-                        .filter((e) -> filter.apply(e.getKey(), e.getValue()))
-                        .map(Map.Entry::getKey)
-                        .collect(Collectors.toList());
-        }
-
-        Set<File> getTemporaryFiles(File file)
-        {
-            try (TransactionData txn = TransactionData.make(file))
-            {
-                maybeFail(txn.readLogFile(null));
-                return txn.getTemporaryFiles();
-            }
-            catch(Throwable t)
-            {
-                // We always fail if the onTxnErr is set to THROW or if we haven't
-                // reached the maximum number of attempts yet. Otherwise
-                // we just log an error and continue as if the txn log file does not exist
-                // clients can choose which behavior they want via onTxnLogError
-                if (attempts < MAX_ATTEMPTS ||
-                    onTxnErr == OnTxnErr.THROW)
-                    throw new RuntimeException(t);
-
-                logger.error("Failed to read temporary files of txn log {}", file, t);
-                return null; // txn.getTemporaryFiles() could be empty so we must use null to differentiate
-            }
-        }
-    }
-
-    @VisibleForTesting
-    static Set<File> getTemporaryFiles(CFMetaData metadata, File folder)
-    {
-        Set<File> ret = new HashSet<>();
-
-        List<File> directories = new Directories(metadata).getCFDirectories();
-        directories.add(folder);
-        for (File dir : directories)
-            ret.addAll(new FileLister(dir.toPath(),
-                                      (file, type) -> type != FileType.FINAL,
-                                      OnTxnErr.IGNORE).list());
-
-        return ret;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 63b8f3e..811f4ef 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -161,7 +161,6 @@ public abstract class SSTable
         return descriptor.ksname;
     }
 
-    @VisibleForTesting
     public List<String> getAllFilePaths()
     {
         List<String> ret = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index b958240..d8ff36a 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -46,7 +46,6 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.db.rows.SliceableUnfilteredRowIterator;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
@@ -1646,7 +1645,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
      * @return true if the this is the first time the file was marked obsolete.  Calling this
      * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
      */
-    public void markObsolete(TransactionLog.SSTableTidier tidier)
+    public void markObsolete(Runnable tidier)
     {
         if (logger.isDebugEnabled())
             logger.debug("Marking {} compacted", getFilename());
@@ -2182,7 +2181,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         // sstable have been released
         private ScheduledFuture readMeterSyncFuture;
         // shared state managing if the logical sstable has been compacted; this is used in cleanup
-        private volatile TransactionLog.SSTableTidier obsoletion;
+        private volatile Runnable obsoletion;
 
         GlobalTidy(final SSTableReader reader)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 920eee0..8b7b1e1 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -25,6 +25,7 @@ import java.nio.charset.StandardCharsets;
 import java.nio.file.*;
 import java.text.DecimalFormat;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import sun.nio.ch.DirectBuffer;
@@ -320,16 +321,16 @@ public class FileUtils
     }
 
     /** Convert absolute path into a path relative to the base path */
-    public static String getRelativePath(String basePath, String absolutePath)
+    public static String getRelativePath(String basePath, String path)
     {
         try
         {
-            return Paths.get(basePath).relativize(Paths.get(absolutePath)).toString();
+            return Paths.get(basePath).relativize(Paths.get(path)).toString();
         }
         catch(Exception ex)
         {
             String absDataPath = FileUtils.getCanonicalPath(basePath);
-            return Paths.get(absDataPath).relativize(Paths.get(absolutePath)).toString();
+            return Paths.get(absDataPath).relativize(Paths.get(path)).toString();
         }
     }
 
@@ -619,6 +620,9 @@ public class FileUtils
         }
         catch (IOException ex)
         {
+            if (ex instanceof NoSuchFileException)
+                return Collections.emptyList();
+
             throw new RuntimeException(ex);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/service/GCInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java
index 252d1c3..21ecdb0 100644
--- a/src/java/org/apache/cassandra/service/GCInspector.java
+++ b/src/java/org/apache/cassandra/service/GCInspector.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
 import com.sun.management.GarbageCollectionNotificationInfo;
 import com.sun.management.GcInfo;
 
-import org.apache.cassandra.db.lifecycle.TransactionLog;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.utils.StatusLogger;
 
 public class GCInspector implements NotificationListener, GCInspectorMXBean
@@ -284,7 +284,7 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
 
             // if we just finished an old gen collection and we're still using a lot of memory, try to reduce the pressure
             if (gcState.assumeGCIsOldGen)
-                TransactionLog.rescheduleFailedDeletions();
+                LifecycleTransaction.rescheduleFailedDeletions();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 2d9bbec..fc2153f 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -60,7 +60,7 @@ import org.apache.cassandra.batchlog.BatchRemoveVerbHandler;
 import org.apache.cassandra.batchlog.BatchlogManager;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.BootStrapper;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
@@ -4262,7 +4262,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public void rescheduleFailedDeletions()
     {
-        TransactionLog.rescheduleFailedDeletions();
+        LifecycleTransaction.rescheduleFailedDeletions();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index f3a1a35..f82cd39 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -34,7 +34,6 @@ import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.*;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -145,7 +144,7 @@ public class StandaloneScrubber
             // Check (and repair) manifests
             checkManifest(cfs.getCompactionStrategyManager(), cfs, sstables);
             CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             System.exit(0); // We need that to stop non daemonized threads
         }
         catch (Exception e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
index a192491..3c4f7dc 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -23,7 +23,6 @@ import java.util.*;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
@@ -164,7 +163,7 @@ public class StandaloneSplitter
                 }
             }
             CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             System.exit(0); // We need that to stop non daemonized threads
         }
         catch (Exception e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index cf94c99..a868917 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.tools;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
@@ -120,7 +119,7 @@ public class StandaloneUpgrader
                 }
             }
             CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             System.exit(0);
         }
         catch (Exception e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/utils/CLibrary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CLibrary.java b/src/java/org/apache/cassandra/utils/CLibrary.java
index 645a4ce..f5da13a 100644
--- a/src/java/org/apache/cassandra/utils/CLibrary.java
+++ b/src/java/org/apache/cassandra/utils/CLibrary.java
@@ -264,7 +264,7 @@ public final class CLibrary
             if (!(e instanceof LastErrorException))
                 throw e;
 
-            logger.warn(String.format("fsync(%d) failed, errno (%d).", fd, errno(e)));
+            logger.warn(String.format("fsync(%d) failed, errno (%d) {}", fd, errno(e)), e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index aedba0e..569451b 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -32,12 +32,11 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.KeyCacheKey;
-import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -164,7 +163,7 @@ public class KeyCacheTest
 
         refs.release();
 
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
 
         // after releasing the reference this should drop to 2
         assertKeyCacheSize(2, KEYSPACE1, COLUMN_FAMILY1);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index e9bf4c5..2fc8436 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -37,7 +37,6 @@ import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.compaction.Scrubber;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
-import org.apache.cassandra.db.lifecycle.TransactionLog;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.db.partitions.Partition;
@@ -370,7 +369,7 @@ public class ScrubTest
             {
                 scrubber.scrub();
             }
-            TransactionLog.waitForDeletions();
+            LifecycleTransaction.waitForDeletions();
             cfs.loadNewSSTables();
             assertOrderedAll(cfs, 7);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
index 7f1b2bd..0488245 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
@@ -159,10 +159,10 @@ public class HelpersTest
     public void testMarkObsolete()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        TransactionLog txnLogs = new TransactionLog(OperationType.UNKNOWN, cfs.metadata);
+        LogTransaction txnLogs = new LogTransaction(OperationType.UNKNOWN, cfs.metadata);
         Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
 
-        List<TransactionLog.Obsoletion> obsoletions = new ArrayList<>();
+        List<LogTransaction.Obsoletion> obsoletions = new ArrayList<>();
         Assert.assertNull(Helpers.prepareForObsoletion(readers, txnLogs, obsoletions, null));
         assertNotNull(obsoletions);
         assertEquals(2, obsoletions.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index db27662..6435e3e 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -249,7 +249,7 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
 
     protected TestableTransaction newTest()
     {
-        TransactionLog.waitForDeletions();
+        LogTransaction.waitForDeletions();
         SSTableReader.resetTidying();
         return new TxnTest();
     }


[3/7] cassandra git commit: Handle non-atomic directory streams safely (CASSANDRA-10109)

Posted by be...@apache.org.
Handle non-atomic directory streams safely (CASSANDRA-10109)

This patch refactors the lifecycle transaction log and updates
the logic to be robust to non-atomic listings of directories

patch by stefania; reviewed by benedict for CASSANDRA-10109


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/351c7cac
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/351c7cac
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/351c7cac

Branch: refs/heads/cassandra-3.0
Commit: 351c7caca311834f6c5bff08b0204943850214a9
Parents: 3818d30
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Thu Aug 27 14:09:45 2015 +0800
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Tue Sep 8 11:53:22 2015 +0100

----------------------------------------------------------------------
 .../apache/cassandra/db/ColumnFamilyStore.java  |    2 +-
 .../org/apache/cassandra/db/Directories.java    |    5 +-
 .../apache/cassandra/db/lifecycle/Helpers.java  |   12 +-
 .../db/lifecycle/LifecycleTransaction.java      |   64 +-
 .../db/lifecycle/LogAwareFileLister.java        |  196 +++
 .../apache/cassandra/db/lifecycle/LogFile.java  |  364 ++++++
 .../cassandra/db/lifecycle/LogRecord.java       |  208 ++++
 .../cassandra/db/lifecycle/LogTransaction.java  |  418 +++++++
 .../apache/cassandra/db/lifecycle/Tracker.java  |    4 +-
 .../cassandra/db/lifecycle/TransactionLog.java  | 1141 ------------------
 .../apache/cassandra/io/sstable/SSTable.java    |    1 -
 .../io/sstable/format/SSTableReader.java        |    5 +-
 .../org/apache/cassandra/io/util/FileUtils.java |   10 +-
 .../apache/cassandra/service/GCInspector.java   |    4 +-
 .../cassandra/service/StorageService.java       |    4 +-
 .../cassandra/tools/StandaloneScrubber.java     |    3 +-
 .../cassandra/tools/StandaloneSplitter.java     |    3 +-
 .../cassandra/tools/StandaloneUpgrader.java     |    3 +-
 .../org/apache/cassandra/utils/CLibrary.java    |    2 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |    5 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |    3 +-
 .../cassandra/db/lifecycle/HelpersTest.java     |    4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |    2 +-
 .../db/lifecycle/LogTransactionTest.java        |  823 +++++++++++++
 .../db/lifecycle/RealTransactionsTest.java      |    7 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |    7 +-
 .../db/lifecycle/TransactionLogTest.java        |  812 -------------
 .../io/sstable/SSTableRewriterTest.java         |   35 +-
 .../org/apache/cassandra/schema/DefsTest.java   |    4 +-
 29 files changed, 2109 insertions(+), 2042 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 096172d..979e8ba 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -510,7 +510,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         SystemKeyspace.removeTruncationRecord(metadata.cfId);
 
         data.dropSSTables();
-        TransactionLog.waitForDeletions();
+        LifecycleTransaction.waitForDeletions();
         indexManager.invalidateAllIndexesBlocking();
         materializedViewManager.invalidate();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index c17b1fd..c801952 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -32,6 +32,7 @@ import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BiFunction;
+import java.util.function.Consumer;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
@@ -550,7 +551,7 @@ public class Directories
         TEMPORARY,
 
         /** A transaction log file (contains information on final and temporary files). */
-        TXN_LOG
+        TXN_LOG;
     }
 
     /**
@@ -562,7 +563,7 @@ public class Directories
         /** Throw the exception */
         THROW,
 
-        /** Ignore the txn log file */
+        /** Ignore the problematic parts of the txn log file */
         IGNORE
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
index 98983c5..f9555f4 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
@@ -106,12 +106,12 @@ class Helpers
             assert !reader.isReplaced();
     }
 
-    static Throwable markObsolete(List<TransactionLog.Obsoletion> obsoletions, Throwable accumulate)
+    static Throwable markObsolete(List<LogTransaction.Obsoletion> obsoletions, Throwable accumulate)
     {
         if (obsoletions == null || obsoletions.isEmpty())
             return accumulate;
 
-        for (TransactionLog.Obsoletion obsoletion : obsoletions)
+        for (LogTransaction.Obsoletion obsoletion : obsoletions)
         {
             try
             {
@@ -125,13 +125,13 @@ class Helpers
         return accumulate;
     }
 
-    static Throwable prepareForObsoletion(Iterable<SSTableReader> readers, TransactionLog txnLogs, List<TransactionLog.Obsoletion> obsoletions, Throwable accumulate)
+    static Throwable prepareForObsoletion(Iterable<SSTableReader> readers, LogTransaction txnLogs, List<LogTransaction.Obsoletion> obsoletions, Throwable accumulate)
     {
         for (SSTableReader reader : readers)
         {
             try
             {
-                obsoletions.add(new TransactionLog.Obsoletion(reader, txnLogs.obsoleted(reader)));
+                obsoletions.add(new LogTransaction.Obsoletion(reader, txnLogs.obsoleted(reader)));
             }
             catch (Throwable t)
             {
@@ -141,12 +141,12 @@ class Helpers
         return accumulate;
     }
 
-    static Throwable abortObsoletion(List<TransactionLog.Obsoletion> obsoletions, Throwable accumulate)
+    static Throwable abortObsoletion(List<LogTransaction.Obsoletion> obsoletions, Throwable accumulate)
     {
         if (obsoletions == null || obsoletions.isEmpty())
             return accumulate;
 
-        for (TransactionLog.Obsoletion obsoletion : obsoletions)
+        for (LogTransaction.Obsoletion obsoletion : obsoletions)
         {
             try
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
index 520b229..59bbc7d 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -98,7 +98,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
 
     public final Tracker tracker;
     // The transaction logs keep track of new and old sstable files
-    private final TransactionLog transactionLog;
+    private final LogTransaction log;
     // the original readers this transaction was opened over, and that it guards
     // (no other transactions may operate over these readers concurrently)
     private final Set<SSTableReader> originals = new HashSet<>();
@@ -115,7 +115,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
     private final State staged = new State();
 
     // the tidier and their readers, to be used for marking readers obsoleted during a commit
-    private List<TransactionLog.Obsoletion> obsoletions;
+    private List<LogTransaction.Obsoletion> obsoletions;
 
     /**
      * construct a Transaction for use in an offline operation
@@ -143,7 +143,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
     public static LifecycleTransaction offline(OperationType operationType, CFMetaData metadata)
     {
         Tracker dummy = new Tracker(null, false);
-        return new LifecycleTransaction(dummy, new TransactionLog(operationType, metadata, dummy), Collections.emptyList());
+        return new LifecycleTransaction(dummy, new LogTransaction(operationType, metadata, dummy), Collections.emptyList());
     }
 
     /**
@@ -152,18 +152,18 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
     public static LifecycleTransaction offline(OperationType operationType, File operationFolder)
     {
         Tracker dummy = new Tracker(null, false);
-        return new LifecycleTransaction(dummy, new TransactionLog(operationType, operationFolder, dummy), Collections.emptyList());
+        return new LifecycleTransaction(dummy, new LogTransaction(operationType, operationFolder, dummy), Collections.emptyList());
     }
 
     LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable<SSTableReader> readers)
     {
-        this(tracker, new TransactionLog(operationType, getMetadata(tracker, readers), tracker), readers);
+        this(tracker, new LogTransaction(operationType, getMetadata(tracker, readers), tracker), readers);
     }
 
-    LifecycleTransaction(Tracker tracker, TransactionLog transactionLog, Iterable<SSTableReader> readers)
+    LifecycleTransaction(Tracker tracker, LogTransaction log, Iterable<SSTableReader> readers)
     {
         this.tracker = tracker;
-        this.transactionLog = transactionLog;
+        this.log = log;
         for (SSTableReader reader : readers)
         {
             originals.add(reader);
@@ -187,19 +187,19 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         return null;
     }
 
-    public TransactionLog log()
+    public LogTransaction log()
     {
-        return transactionLog;
+        return log;
     }
 
     public OperationType opType()
     {
-        return transactionLog.getType();
+        return log.getType();
     }
 
     public UUID opId()
     {
-        return transactionLog.getId();
+        return log.getId();
     }
 
     public void doPrepare()
@@ -212,8 +212,8 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
 
         // prepare for compaction obsolete readers as long as they were part of the original set
         // since those that are not original are early readers that share the same desc with the finals
-        maybeFail(prepareForObsoletion(filterIn(logged.obsolete, originals), transactionLog, obsoletions = new ArrayList<>(), null));
-        transactionLog.prepareToCommit();
+        maybeFail(prepareForObsoletion(filterIn(logged.obsolete, originals), log, obsoletions = new ArrayList<>(), null));
+        log.prepareToCommit();
     }
 
     /**
@@ -228,7 +228,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         maybeFail(accumulate);
 
         // transaction log commit failure means we must abort; safe commit is not possible
-        maybeFail(transactionLog.commit(null));
+        maybeFail(log.commit(null));
 
         // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done
         // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size
@@ -237,7 +237,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         accumulate = markObsolete(obsoletions, accumulate);
         accumulate = tracker.updateSizeTracking(logged.obsolete, logged.update, accumulate);
         accumulate = release(selfRefs(logged.obsolete), accumulate);
-        accumulate = tracker.notifySSTablesChanged(originals, logged.update, transactionLog.getType(), accumulate);
+        accumulate = tracker.notifySSTablesChanged(originals, logged.update, log.getType(), accumulate);
 
         return accumulate;
     }
@@ -253,16 +253,16 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         accumulate = abortObsoletion(obsoletions, accumulate);
 
         if (logged.isEmpty() && staged.isEmpty())
-            return transactionLog.abort(accumulate);
+            return log.abort(accumulate);
 
         // mark obsolete all readers that are not versions of those present in the original set
         Iterable<SSTableReader> obsolete = filterOut(concatUniq(staged.update, logged.update), originals);
         logger.debug("Obsoleting {}", obsolete);
 
-        accumulate = prepareForObsoletion(obsolete, transactionLog, obsoletions = new ArrayList<>(), accumulate);
+        accumulate = prepareForObsoletion(obsolete, log, obsoletions = new ArrayList<>(), accumulate);
         // it's safe to abort even if committed, see maybeFail in doCommit() above, in this case it will just report
         // a failure to abort, which is useful information to have for debug
-        accumulate = transactionLog.abort(accumulate);
+        accumulate = log.abort(accumulate);
         accumulate = markObsolete(obsoletions, accumulate);
 
         // replace all updated readers with a version restored to its original state
@@ -502,7 +502,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
             originals.remove(reader);
             marked.remove(reader);
         }
-        return new LifecycleTransaction(tracker, transactionLog.getType(), readers);
+        return new LifecycleTransaction(tracker, log.getType(), readers);
     }
 
     /**
@@ -535,17 +535,17 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
 
     public void trackNew(SSTable table)
     {
-        transactionLog.trackNew(table);
+        log.trackNew(table);
     }
 
     public void untrackNew(SSTable table)
     {
-        transactionLog.untrackNew(table);
+        log.untrackNew(table);
     }
 
     public static void removeUnfinishedLeftovers(CFMetaData metadata)
     {
-        TransactionLog.removeUnfinishedLeftovers(metadata);
+        LogTransaction.removeUnfinishedLeftovers(metadata);
     }
 
     /**
@@ -562,7 +562,25 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
      */
     public static List<File> getFiles(Path folder, BiFunction<File, Directories.FileType, Boolean> filter, Directories.OnTxnErr onTxnErr)
     {
-        return new TransactionLog.FileLister(folder, filter, onTxnErr).list();
+        return new LogAwareFileLister(folder, filter, onTxnErr).list();
+    }
+
+    /**
+     * Retry all deletions that failed the first time around (presumably b/c the sstable was still mmap'd.)
+     * Useful because there are times when we know GC has been invoked; also exposed as an mbean.
+     */
+    public static void rescheduleFailedDeletions()
+    {
+        LogTransaction.rescheduleFailedDeletions();
+    }
+
+    /**
+     * Deletions run on the nonPeriodicTasks executor, (both failedDeletions or global tidiers in SSTableReader)
+     * so by scheduling a new empty task and waiting for it we ensure any prior deletion has completed.
+     */
+    public static void waitForDeletions()
+    {
+        LogTransaction.waitForDeletions();
     }
 
     // a class representing the current state of the reader within this transaction, encoding the actions both logged

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java b/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
new file mode 100644
index 0000000..e086078
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
@@ -0,0 +1,196 @@
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.*;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.Directories;
+
+import static org.apache.cassandra.db.Directories.*;
+
+/**
+ * A class for listing files in a folder.
+ */
+final class LogAwareFileLister
+{
+    // The folder to scan
+    private final Path folder;
+
+    // The filter determines which files the client wants returned
+    private final BiFunction<File, FileType, Boolean> filter; //file, file type
+
+    // The behavior when we fail to list files
+    private final OnTxnErr onTxnErr;
+
+    // The unfiltered result
+    NavigableMap<File, Directories.FileType> files = new TreeMap<>();
+
+    @VisibleForTesting
+    LogAwareFileLister(Path folder, BiFunction<File, FileType, Boolean> filter, OnTxnErr onTxnErr)
+    {
+        this.folder = folder;
+        this.filter = filter;
+        this.onTxnErr = onTxnErr;
+    }
+
+    public List<File> list()
+    {
+        try
+        {
+            return innerList();
+        }
+        catch (Throwable t)
+        {
+            throw new RuntimeException(String.format("Failed to list files in %s", folder), t);
+        }
+    }
+
+    List<File> innerList() throws Throwable
+    {
+        list(Files.newDirectoryStream(folder))
+        .stream()
+        .filter((f) -> !LogFile.isLogFile(f))
+        .forEach((f) -> files.put(f, FileType.FINAL));
+
+        // Since many file systems are not atomic, we cannot be sure we have listed a consistent disk state
+        // (Linux would permit this, but for simplicity we keep our behaviour the same across platforms)
+        // so we must be careful to list txn log files AFTER every other file since these files are deleted last,
+        // after all other files are removed
+        list(Files.newDirectoryStream(folder, '*' + LogFile.EXT))
+        .stream()
+        .filter(LogFile::isLogFile)
+        .forEach(this::classifyFiles);
+
+        // Finally we apply the user filter before returning our result
+        return files.entrySet().stream()
+                    .filter((e) -> filter.apply(e.getKey(), e.getValue()))
+                    .map(Map.Entry::getKey)
+                    .collect(Collectors.toList());
+    }
+
+    static List<File> list(DirectoryStream<Path> stream) throws IOException
+    {
+        try
+        {
+            return StreamSupport.stream(stream.spliterator(), false)
+                                .map(Path::toFile)
+                                .filter((f) -> !f.isDirectory())
+                                .collect(Collectors.toList());
+        }
+        finally
+        {
+            stream.close();
+        }
+    }
+
+    /**
+     * We read txn log files, if we fail we throw only if the user has specified
+     * OnTxnErr.THROW, else we log an error and apply the txn log anyway
+     */
+    void classifyFiles(File txnFile)
+    {
+        LogFile txn = LogFile.make(txnFile, -1);
+        readTxnLog(txn);
+        classifyFiles(txn);
+        files.put(txnFile, FileType.TXN_LOG);
+    }
+
+    void readTxnLog(LogFile txn)
+    {
+        txn.readRecords();
+        if (!txn.verify() && onTxnErr == OnTxnErr.THROW)
+            throw new LogTransaction.CorruptTransactionLogException("Some records failed verification. See earlier in log for details.", txn);
+    }
+
+    void classifyFiles(LogFile txnFile)
+    {
+        Map<LogRecord, Set<File>> oldFiles = txnFile.getFilesOfType(files.navigableKeySet(), LogRecord.Type.REMOVE);
+        Map<LogRecord, Set<File>> newFiles = txnFile.getFilesOfType(files.navigableKeySet(), LogRecord.Type.ADD);
+
+        if (txnFile.completed())
+        { // last record present, filter regardless of disk status
+            setTemporary(txnFile, oldFiles.values(), newFiles.values());
+            return;
+        }
+
+        if (allFilesPresent(txnFile, oldFiles, newFiles))
+        {  // all files present, transaction is in progress, this will filter as aborted
+            setTemporary(txnFile, oldFiles.values(), newFiles.values());
+            return;
+        }
+
+        // some files are missing, we expect the txn file to either also be missing or completed, so check
+        // disk state again to resolve any previous races on non-atomic directory listing platforms
+
+        // if txn file also gone, then do nothing (all temporary should be gone, we could remove them if any)
+        if (!txnFile.exists())
+            return;
+
+        // otherwise read the file again to see if it is completed now
+        readTxnLog(txnFile);
+
+        if (txnFile.completed())
+        { // if after re-reading the txn is completed then filter accordingly
+            setTemporary(txnFile, oldFiles.values(), newFiles.values());
+            return;
+        }
+
+        // some files are missing and yet the txn is still there and not completed
+        // something must be wrong (see comment at the top of this file requiring txn to be
+        // completed before obsoleting or aborting sstables)
+        throw new RuntimeException(String.format("Failed to list directory files in %s, inconsistent disk state for transaction %s",
+                                                 folder,
+                                                 txnFile));
+    }
+
+    /** See if all files are present or if only the last record files are missing and it's a NEW record */
+    private static boolean allFilesPresent(LogFile txnFile, Map<LogRecord, Set<File>> oldFiles, Map<LogRecord, Set<File>> newFiles)
+    {
+        LogRecord lastRecord = txnFile.getLastRecord();
+        return !Stream.concat(oldFiles.entrySet().stream(),
+                              newFiles.entrySet().stream()
+                                      .filter((e) -> e.getKey() != lastRecord))
+                      .filter((e) -> e.getKey().numFiles > e.getValue().size())
+                      .findFirst().isPresent();
+    }
+
+    private void setTemporary(LogFile txnFile, Collection<Set<File>> oldFiles, Collection<Set<File>> newFiles)
+    {
+        Collection<Set<File>> temporary = txnFile.committed() ? oldFiles : newFiles;
+        temporary.stream()
+                 .flatMap(Set::stream)
+                 .forEach((f) -> this.files.put(f, FileType.TEMPORARY));
+    }
+
+    @VisibleForTesting
+    static Set<File> getTemporaryFiles(File folder)
+    {
+        return listFiles(folder, FileType.TEMPORARY);
+    }
+
+    @VisibleForTesting
+    static Set<File> getFinalFiles(File folder)
+    {
+        return listFiles(folder, FileType.FINAL);
+    }
+
+    @VisibleForTesting
+    static Set<File> listFiles(File folder, FileType ... types)
+    {
+        Collection<FileType> match = Arrays.asList(types);
+        return new LogAwareFileLister(folder.toPath(),
+                                      (file, type) -> match.contains(type),
+                                      OnTxnErr.IGNORE).list()
+                                                      .stream()
+                                                      .collect(Collectors.toSet());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogFile.java b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
new file mode 100644
index 0000000..c698722
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
@@ -0,0 +1,364 @@
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LogRecord.Type;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.big.BigFormat;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.CLibrary;
+
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * The transaction log file, which contains many records.
+ */
+final class LogFile
+{
+    private static final Logger logger = LoggerFactory.getLogger(LogFile.class);
+
+    static String EXT = ".log";
+    static char SEP = '_';
+    // cc_txn_opname_id.log (where cc is one of the sstable versions defined in BigVersion)
+    static Pattern FILE_REGEX = Pattern.compile(String.format("^(.{2})_txn_(.*)_(.*)%s$", EXT));
+
+    final File file;
+    final Set<LogRecord> records = new LinkedHashSet<>();
+    final OperationType opType;
+    final UUID id;
+    final File folder;
+    final int folderDescriptor;
+
+    static LogFile make(File logFile, int folderDescriptor)
+    {
+        Matcher matcher = LogFile.FILE_REGEX.matcher(logFile.getName());
+        assert matcher.matches() && matcher.groupCount() == 3;
+
+        // For now we don't need this but it is there in case we need to change
+        // file format later on, the version is the sstable version as defined in BigFormat
+        //String version = matcher.group(1);
+
+        OperationType operationType = OperationType.fromFileName(matcher.group(2));
+        UUID id = UUID.fromString(matcher.group(3));
+
+        return new LogFile(operationType, logFile.getParentFile(), folderDescriptor, id);
+    }
+
+    void sync()
+    {
+        if (folderDescriptor > 0)
+            CLibrary.trySync(folderDescriptor);
+    }
+
+    OperationType getType()
+    {
+        return opType;
+    }
+
+    UUID getId()
+    {
+        return id;
+    }
+
+    Throwable removeUnfinishedLeftovers(Throwable accumulate)
+    {
+        try
+        {
+            deleteRecords(committed() ? Type.REMOVE : Type.ADD);
+
+            // we sync the parent file descriptor between contents and log deletion
+            // to ensure there is a happens before edge between them
+            sync();
+
+            Files.delete(file.toPath());
+        }
+        catch (Throwable t)
+        {
+            accumulate = merge(accumulate, t);
+        }
+
+        return accumulate;
+    }
+
+    static boolean isLogFile(File file)
+    {
+        return LogFile.FILE_REGEX.matcher(file.getName()).matches();
+    }
+
+    LogFile(OperationType opType, File folder, int folderDescriptor, UUID id)
+    {
+        this.opType = opType;
+        this.id = id;
+        this.folder = folder;
+        this.file = new File(getFileName(folder, opType, id));
+        this.folderDescriptor = folderDescriptor;
+    }
+
+    public void readRecords()
+    {
+        assert records.isEmpty();
+        FileUtils.readLines(file).stream()
+                 .map(LogRecord::make)
+                 .forEach(records::add);
+    }
+
+    public boolean verify()
+    {
+        Optional<LogRecord> firstInvalid = records.stream()
+                                                  .filter(this::isInvalid)
+                                                  .findFirst();
+
+        if (!firstInvalid.isPresent())
+            return true;
+
+        LogRecord failedOn = firstInvalid.get();
+        if (getLastRecord() != failedOn)
+        {
+            logError(failedOn);
+            return false;
+        }
+
+        if (records.stream()
+                   .filter((r) -> r != failedOn)
+                   .filter(LogFile::isInvalidWithCorruptedLastRecord)
+                   .map(LogFile::logError)
+                   .findFirst().isPresent())
+        {
+            logError(failedOn);
+            return false;
+        }
+
+        // if only the last record is corrupt and all other records have matching files on disk, @see verifyRecord,
+        // then we simply exited whilst serializing the last record and we carry on
+        logger.warn(String.format("Last record of transaction %s is corrupt or incomplete [%s], but all previous records match state on disk; continuing",
+                                  id,
+                                  failedOn.error));
+        return true;
+    }
+
+    static LogRecord logError(LogRecord record)
+    {
+        logger.error("{}", record.error);
+        return record;
+    }
+
+    boolean isInvalid(LogRecord record)
+    {
+        if (!record.isValid())
+            return true;
+
+        if (record.type == Type.UNKNOWN)
+        {
+            record.error(String.format("Could not parse record [%s]", record));
+            return true;
+        }
+
+        if (record.checksum != record.computeChecksum())
+        {
+            record.error(String.format("Invalid checksum for sstable [%s], record [%s]: [%d] should have been [%d]",
+                                       record.relativeFilePath,
+                                       record,
+                                       record.checksum,
+                                       record.computeChecksum()));
+            return true;
+        }
+
+        if (record.type != Type.REMOVE)
+            return false;
+
+        List<File> files = record.getExistingFiles(folder);
+
+        // Paranoid sanity checks: we create another record by looking at the files as they are
+        // on disk right now and make sure the information still matches
+        record.onDiskRecord = LogRecord.make(record.type, files, 0, record.relativeFilePath);
+
+        if (record.updateTime != record.onDiskRecord.updateTime && record.onDiskRecord.numFiles > 0)
+        {
+            record.error(String.format("Unexpected files detected for sstable [%s], record [%s]: last update time [%tT] should have been [%tT]",
+                                       record.relativeFilePath,
+                                       record,
+                                       record.onDiskRecord.updateTime,
+                                       record.updateTime));
+            return true;
+        }
+
+        return false;
+    }
+
+    static boolean isInvalidWithCorruptedLastRecord(LogRecord record)
+    {
+        if (record.type == Type.REMOVE && record.onDiskRecord.numFiles < record.numFiles)
+        { // if we found a corruption in the last record, then we continue only if the number of files matches exactly for all previous records.
+            record.error(String.format("Incomplete fileset detected for sstable [%s], record [%s]: number of files [%d] should have been [%d]. Treating as unrecoverable due to corruption of the final record.",
+                         record.relativeFilePath,
+                         record.raw,
+                         record.onDiskRecord.numFiles,
+                         record.numFiles));
+            return true;
+        }
+        return false;
+    }
+
+    public void commit()
+    {
+        assert !completed() : "Already completed!";
+        addRecord(LogRecord.makeCommit(System.currentTimeMillis()));
+    }
+
+    public void abort()
+    {
+        assert !completed() : "Already completed!";
+        addRecord(LogRecord.makeAbort(System.currentTimeMillis()));
+    }
+
+    private boolean isLastRecordValidWithType(Type type)
+    {
+        LogRecord lastRecord = getLastRecord();
+        return lastRecord != null &&
+               lastRecord.type == type &&
+               !isInvalid(lastRecord);
+    }
+
+    public boolean committed()
+    {
+        return isLastRecordValidWithType(Type.COMMIT);
+    }
+
+    public boolean aborted()
+    {
+        return isLastRecordValidWithType(Type.ABORT);
+    }
+
+    public boolean completed()
+    {
+        return committed() || aborted();
+    }
+
+    public void add(Type type, SSTable table)
+    {
+        if (!addRecord(makeRecord(type, table)))
+            throw new IllegalStateException();
+    }
+
+    private LogRecord makeRecord(Type type, SSTable table)
+    {
+        assert type == Type.ADD || type == Type.REMOVE;
+        return LogRecord.make(type, folder, table);
+    }
+
+    private boolean addRecord(LogRecord record)
+    {
+        if (!records.add(record))
+            return false;
+
+        // we only checksum the records, not the checksums themselves
+        FileUtils.append(file, record.toString());
+        sync();
+        return true;
+    }
+
+    public void remove(Type type, SSTable table)
+    {
+        LogRecord record = makeRecord(type, table);
+
+        assert records.contains(record) : String.format("[%s] is not tracked by %s", record, file);
+
+        records.remove(record);
+        deleteRecord(record);
+    }
+
+    public boolean contains(Type type, SSTable table)
+    {
+        return records.contains(makeRecord(type, table));
+    }
+
+    public void deleteRecords(Type type)
+    {
+        assert file.exists() : String.format("Expected %s to exists", file);
+        records.stream()
+               .filter(type::matches)
+               .forEach(this::deleteRecord);
+        records.clear();
+    }
+
+    private void deleteRecord(LogRecord record)
+    {
+        List<File> files = record.getExistingFiles(folder);
+
+        // we sort the files in ascending update time order so that the last update time
+        // stays the same even if we only partially delete files
+        files.sort((f1, f2) -> Long.compare(f1.lastModified(), f2.lastModified()));
+
+        files.forEach(LogTransaction::delete);
+    }
+
+    public Map<LogRecord, Set<File>> getFilesOfType(NavigableSet<File> files, Type type)
+    {
+        Map<LogRecord, Set<File>> ret = new HashMap<>();
+
+        records.stream()
+               .filter(type::matches)
+               .filter(LogRecord::isValid)
+               .forEach((r) -> ret.put(r, getRecordFiles(files, r)));
+
+        return ret;
+    }
+
+    public LogRecord getLastRecord()
+    {
+        return Iterables.getLast(records, null);
+    }
+
+    private Set<File> getRecordFiles(NavigableSet<File> files, LogRecord record)
+    {
+        Set<File> ret = new HashSet<>();
+        for (File file : files.tailSet(new File(folder, record.relativeFilePath)))
+        {
+            if (!file.getName().startsWith(record.relativeFilePath))
+                break;
+            ret.add(file);
+        }
+        return ret;
+    }
+
+    public void delete()
+    {
+        LogTransaction.delete(file);
+    }
+
+    public boolean exists()
+    {
+        return file.exists();
+    }
+
+    @Override
+    public String toString()
+    {
+        return FileUtils.getRelativePath(folder.getPath(), file.getPath());
+    }
+
+    static String getFileName(File folder, OperationType opType, UUID id)
+    {
+        String fileName = StringUtils.join(BigFormat.latestVersion,
+                                           LogFile.SEP,
+                                           "txn",
+                                           LogFile.SEP,
+                                           opType.fileName,
+                                           LogFile.SEP,
+                                           id.toString(),
+                                           LogFile.EXT);
+        return StringUtils.join(folder, File.separator, fileName);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
new file mode 100644
index 0000000..0f0f3a2
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
@@ -0,0 +1,208 @@
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.zip.CRC32;
+
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * A log file record, each record is encoded in one line and has different
+ * content depending on the record type.
+ */
+final class LogRecord
+{
+    public enum Type
+    {
+        UNKNOWN, // a record that cannot be parsed
+        ADD,    // new files to be retained on commit
+        REMOVE, // old files to be retained on abort
+        COMMIT, // commit flag
+        ABORT;  // abort flag
+
+        public static Type fromPrefix(String prefix)
+        {
+            return valueOf(prefix.toUpperCase());
+        }
+
+        public boolean hasFile()
+        {
+            return this == Type.ADD || this == Type.REMOVE;
+        }
+
+        public boolean matches(LogRecord record)
+        {
+            return this == record.type;
+        }
+    }
+
+
+    public final Type type;
+    public final String relativeFilePath;
+    public final long updateTime;
+    public final int numFiles;
+    public final String raw;
+    public final long checksum;
+
+    public String error;
+    public LogRecord onDiskRecord;
+
+    // (add|remove|commit|abort):[*,*,*][checksum]
+    static Pattern REGEX = Pattern.compile("^(add|remove|commit|abort):\\[([^,]*),?([^,]*),?([^,]*)\\]\\[(\\d*)\\]$", Pattern.CASE_INSENSITIVE);
+
+    public static LogRecord make(String line)
+    {
+        try
+        {
+            Matcher matcher = REGEX.matcher(line);
+            if (!matcher.matches())
+                return new LogRecord(Type.UNKNOWN, "", 0, 0, 0, line)
+                       .error(String.format("Failed to parse [%s]", line));
+
+            Type type = Type.fromPrefix(matcher.group(1));
+            return new LogRecord(type, matcher.group(2), Long.valueOf(matcher.group(3)), Integer.valueOf(matcher.group(4)), Long.valueOf(matcher.group(5)), line);
+        }
+        catch (Throwable t)
+        {
+            return new LogRecord(Type.UNKNOWN, "", 0, 0, 0, line).error(t);
+        }
+    }
+
+    public static LogRecord makeCommit(long updateTime)
+    {
+        return new LogRecord(Type.COMMIT, "", updateTime, 0);
+    }
+
+    public static LogRecord makeAbort(long updateTime)
+    {
+        return new LogRecord(Type.ABORT, "", updateTime, 0);
+    }
+
+    public static LogRecord make(Type type, File parentFolder, SSTable table)
+    {
+        String relativePath = FileUtils.getRelativePath(parentFolder.getPath(), table.descriptor.baseFilename());
+        // why do we take the max of files.size() and table.getAllFilePaths().size()?
+        return make(type, getExistingFiles(parentFolder, relativePath), table.getAllFilePaths().size(), relativePath);
+    }
+
+    public static LogRecord make(Type type, List<File> files, int minFiles, String relativeFilePath)
+    {
+        long lastModified = files.stream().map(File::lastModified).reduce(0L, Long::max);
+        return new LogRecord(type, relativeFilePath, lastModified, Math.max(minFiles, files.size()));
+    }
+
+    private LogRecord(Type type,
+                      String relativeFilePath,
+                      long updateTime,
+                      int numFiles)
+    {
+        this(type, relativeFilePath, updateTime, numFiles, 0, null);
+    }
+
+    private LogRecord(Type type,
+                      String relativeFilePath,
+                      long updateTime,
+                      int numFiles,
+                      long checksum,
+                      String raw)
+    {
+        this.type = type;
+        this.relativeFilePath = type.hasFile() ? relativeFilePath : ""; // only meaningful for file records
+        this.updateTime = type == Type.REMOVE ? updateTime : 0; // only meaningful for old records
+        this.numFiles = type.hasFile() ? numFiles : 0; // only meaningful for file records
+        if (raw == null)
+        {
+            assert checksum == 0;
+            this.checksum = computeChecksum();
+            this.raw = format();
+        }
+        else
+        {
+            this.checksum = checksum;
+            this.raw = raw;
+        }
+
+        this.error = "";
+    }
+
+    public LogRecord error(Throwable t)
+    {
+        return error(t.getMessage());
+    }
+
+    public LogRecord error(String error)
+    {
+        this.error = error;
+        return this;
+    }
+
+    public boolean isValid()
+    {
+        return this.error.isEmpty();
+    }
+
+    private String format()
+    {
+        return String.format("%s:[%s,%d,%d][%d]", type.toString(), relativeFilePath, updateTime, numFiles, checksum);
+    }
+
+    public List<File> getExistingFiles(File folder)
+    {
+        if (!type.hasFile())
+            return Collections.emptyList();
+
+        return getExistingFiles(folder, relativeFilePath);
+    }
+
+    public static List<File> getExistingFiles(File parentFolder, String relativeFilePath)
+    {
+        return Arrays.asList(parentFolder.listFiles((dir, name) -> name.startsWith(relativeFilePath)));
+    }
+
+    @Override
+    public int hashCode()
+    {
+        // see comment in equals
+        return Objects.hash(type, relativeFilePath, error);
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (!(obj instanceof LogRecord))
+            return false;
+
+        final LogRecord other = (LogRecord)obj;
+
+        // we exclude on purpose checksum, update time and count as
+        // we don't want duplicated records that differ only by
+        // properties that might change on disk, especially COMMIT records,
+        // there should be only one regardless of update time
+        // however we must compare the error to make sure we have more than
+        // one UNKNOWN record, if we fail to parse more than one
+        return type == other.type &&
+               relativeFilePath.equals(other.relativeFilePath) &&
+               error.equals(other.error);
+    }
+
+    @Override
+    public String toString()
+    {
+        return raw;
+    }
+
+    long computeChecksum()
+    {
+        CRC32 crc32 = new CRC32();
+        crc32.update(relativeFilePath.getBytes(FileUtils.CHARSET));
+        crc32.update(type.toString().getBytes(FileUtils.CHARSET));
+        FBUtilities.updateChecksumInt(crc32, (int) updateTime);
+        FBUtilities.updateChecksumInt(crc32, (int) (updateTime >>> 32));
+        FBUtilities.updateChecksumInt(crc32, numFiles);
+        return crc32.getValue() & (Long.MAX_VALUE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
new file mode 100644
index 0000000..89d7beb
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@ -0,0 +1,418 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.util.*;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.Runnables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LogRecord.Type;
+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.SnapshotDeletingTask;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.RefCounted;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+/**
+ * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
+ * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
+ * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
+ * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
+ *
+ * IMPORTANT: The transaction must complete (commit or abort) before any temporary files are deleted, even though the
+ * txn log file itself will not be deleted until all tracked files are deleted. This is required by FileLister to ensure
+ * a consistent disk state. LifecycleTransaction ensures this requirement, so this class should really never be used
+ * outside of LT. @see FileLister.classifyFiles(TransactionData txn)
+ *
+ * A class that tracks sstable files involved in a transaction across sstables:
+ * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
+ *
+ * The transaction log file contains new and old sstables as follows:
+ *
+ * add:[sstable-2][CRC]
+ * remove:[sstable-1,max_update_time,num files][CRC]
+ *
+ * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
+ * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
+ * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
+ * and file sizes.
+ *
+ * Upon commit we add a final line to the log file:
+ *
+ * commit:[commit_time][CRC]
+ *
+ * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
+ * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
+ * was committed, vice-versa if the transaction was aborted.
+ *
+ * On start-up we look for any transaction log files and repeat the cleanup process described above.
+ *
+ * See CASSANDRA-7066 for full details.
+ */
+class LogTransaction extends Transactional.AbstractTransactional implements Transactional
+{
+    private static final Logger logger = LoggerFactory.getLogger(LogTransaction.class);
+
+    /**
+     * If the format of the lines in the transaction log is wrong or the checksum
+     * does not match, then we throw this exception.
+     */
+    public static final class CorruptTransactionLogException extends RuntimeException
+    {
+        public final LogFile file;
+
+        public CorruptTransactionLogException(String message, LogFile file)
+        {
+            super(message);
+            this.file = file;
+        }
+    }
+
+    private final Tracker tracker;
+    private final LogFile data;
+    private final Ref<LogTransaction> selfRef;
+    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
+    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
+    // Additionally, we need to make sure to delete the data file first, so on restart the others
+    // will be recognized as GCable.
+    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
+
+    LogTransaction(OperationType opType, CFMetaData metadata)
+    {
+        this(opType, metadata, null);
+    }
+
+    LogTransaction(OperationType opType, CFMetaData metadata, Tracker tracker)
+    {
+        this(opType, new Directories(metadata), tracker);
+    }
+
+    LogTransaction(OperationType opType, Directories directories, Tracker tracker)
+    {
+        this(opType, directories.getDirectoryForNewSSTables(), tracker);
+    }
+
+    LogTransaction(OperationType opType, File folder, Tracker tracker)
+    {
+        this.tracker = tracker;
+        int folderDescriptor = CLibrary.tryOpenDirectory(folder.getPath());
+        this.data = new LogFile(opType, folder, folderDescriptor, UUIDGen.getTimeUUID());
+        this.selfRef = new Ref<>(this, new TransactionTidier(data, folderDescriptor));
+
+        if (logger.isDebugEnabled())
+            logger.debug("Created transaction logs with id {}", data.id);
+    }
+
+    /**
+     * Track a reader as new.
+     **/
+    void trackNew(SSTable table)
+    {
+        data.add(Type.ADD, table);
+    }
+
+    /**
+     * Stop tracking a reader as new.
+     */
+    void untrackNew(SSTable table)
+    {
+        data.remove(Type.ADD, table);
+    }
+
+    /**
+     * Schedule a reader for deletion as soon as it is fully unreferenced.
+     */
+    SSTableTidier obsoleted(SSTableReader reader)
+    {
+        if (data.contains(Type.ADD, reader))
+        {
+            if (data.contains(Type.REMOVE, reader))
+                throw new IllegalArgumentException();
+
+            return new SSTableTidier(reader, true, this);
+        }
+
+        data.add(Type.REMOVE, reader);
+
+        if (tracker != null)
+            tracker.notifyDeleting(reader);
+
+        return new SSTableTidier(reader, false, this);
+    }
+
+    OperationType getType()
+    {
+        return data.getType();
+    }
+
+    UUID getId()
+    {
+        return data.getId();
+    }
+
+    @VisibleForTesting
+    String getDataFolder()
+    {
+        return data.folder.getPath();
+    }
+
+    @VisibleForTesting
+    LogFile getLogFile()
+    {
+        return data;
+    }
+
+    static void delete(File file)
+    {
+        try
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("Deleting {}", file);
+
+            Files.delete(file.toPath());
+        }
+        catch (NoSuchFileException e)
+        {
+            logger.error("Unable to delete {} as it does not exist", file);
+        }
+        catch (IOException e)
+        {
+            logger.error("Unable to delete {}", file, e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * The transaction tidier.
+     *
+     * When the transaction reference is fully released we try to delete all the obsolete files
+     * depending on the transaction result, as well as the transaction log file.
+     */
+    private static class TransactionTidier implements RefCounted.Tidy, Runnable
+    {
+        private final LogFile data;
+        private final int folderDescriptor;
+
+        TransactionTidier(LogFile data, int folderDescriptor)
+        {
+            this.data = data;
+            this.folderDescriptor = folderDescriptor;
+        }
+
+        public void tidy() throws Exception
+        {
+            run();
+        }
+
+        public String name()
+        {
+            return data.toString();
+        }
+
+        public void run()
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("Removing files for transaction {}", name());
+
+            assert data.completed() : "Expected a completed transaction: " + data;
+
+            Throwable err = data.removeUnfinishedLeftovers(null);
+
+            if (err != null)
+            {
+                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
+                failedDeletions.add(this);
+            }
+            else
+            {
+                if (logger.isDebugEnabled())
+                    logger.debug("Closing file transaction {}", name());
+                CLibrary.tryCloseFD(folderDescriptor);
+            }
+        }
+    }
+
+    static class Obsoletion
+    {
+        final SSTableReader reader;
+        final SSTableTidier tidier;
+
+        Obsoletion(SSTableReader reader, SSTableTidier tidier)
+        {
+            this.reader = reader;
+            this.tidier = tidier;
+        }
+    }
+
+    /**
+     * The SSTableReader tidier. When a reader is fully released and no longer referenced
+     * by any one, we run this. It keeps a reference to the parent transaction and releases
+     * it when done, so that the final transaction cleanup can run when all obsolete readers
+     * are released.
+     */
+    public static class SSTableTidier implements Runnable
+    {
+        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
+        private final Descriptor desc;
+        private final long sizeOnDisk;
+        private final Tracker tracker;
+        private final boolean wasNew;
+        private final Ref<LogTransaction> parentRef;
+
+        public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent)
+        {
+            this.desc = referent.descriptor;
+            this.sizeOnDisk = referent.bytesOnDisk();
+            this.tracker = parent.tracker;
+            this.wasNew = wasNew;
+            this.parentRef = parent.selfRef.tryRef();
+        }
+
+        public void run()
+        {
+            SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+
+            try
+            {
+                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
+                File datafile = new File(desc.filenameFor(Component.DATA));
+
+                delete(datafile);
+                // let the remainder be cleaned up by delete
+                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
+            }
+            catch (Throwable t)
+            {
+                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
+                failedDeletions.add(this);
+                return;
+            }
+
+            if (tracker != null && tracker.cfstore != null && !wasNew)
+                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
+
+            // release the referent to the parent so that the all transaction files can be released
+            parentRef.release();
+        }
+
+        public void abort()
+        {
+            parentRef.release();
+        }
+    }
+
+
+    static void rescheduleFailedDeletions()
+    {
+        Runnable task;
+        while ( null != (task = failedDeletions.poll()))
+            ScheduledExecutors.nonPeriodicTasks.submit(task);
+
+        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
+        SnapshotDeletingTask.rescheduleFailedTasks();
+    }
+
+    static void waitForDeletions()
+    {
+        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
+    }
+
+    @VisibleForTesting
+    Throwable complete(Throwable accumulate)
+    {
+        try
+        {
+            accumulate = selfRef.ensureReleased(accumulate);
+            return accumulate;
+        }
+        catch (Throwable t)
+        {
+            logger.error("Failed to complete file transaction {}", getId(), t);
+            return Throwables.merge(accumulate, t);
+        }
+    }
+
+    protected Throwable doCommit(Throwable accumulate)
+    {
+        data.commit();
+        return complete(accumulate);
+    }
+
+    protected Throwable doAbort(Throwable accumulate)
+    {
+        data.abort();
+        return complete(accumulate);
+    }
+
+    protected void doPrepare() { }
+
+    /**
+     * Called on startup to scan existing folders for any unfinished leftovers of
+     * operations that were ongoing when the process exited. Also called by the standalone
+     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
+     *
+     */
+    static void removeUnfinishedLeftovers(CFMetaData metadata)
+    {
+        for (File dir : new Directories(metadata).getCFDirectories())
+        {
+            int folderDescriptor = CLibrary.tryOpenDirectory(dir.getPath());
+            try
+            {
+                File[] logs = dir.listFiles(LogFile::isLogFile);
+
+                for (File log : logs)
+                {
+                    LogFile data = LogFile.make(log, folderDescriptor);
+                    data.readRecords();
+                    if (data.verify())
+                    {
+                        Throwable failure = data.removeUnfinishedLeftovers(null);
+                        logger.error("Failed to remove unfinished transaction leftovers for log {}", log, failure);
+                    }
+                    else
+                    {
+                        logger.error("Unexpected disk state: failed to read transaction log {}", log);
+                    }
+                }
+            }
+            finally
+            {
+                CLibrary.tryCloseFD(folderDescriptor);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/351c7cac/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index d028493..ffb71ee 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -224,7 +224,7 @@ public class Tracker
      */
     public Throwable dropSSTables(final Predicate<SSTableReader> remove, OperationType operationType, Throwable accumulate)
     {
-        try (TransactionLog txnLogs = new TransactionLog(operationType, cfstore.metadata, this))
+        try (LogTransaction txnLogs = new LogTransaction(operationType, cfstore.metadata, this))
         {
             Pair<View, View> result = apply(view -> {
                 Set<SSTableReader> toremove = copyOf(filter(view.sstables, and(remove, notIn(view.compacting))));
@@ -236,7 +236,7 @@ public class Tracker
 
             // It is important that any method accepting/returning a Throwable never throws an exception, and does its best
             // to complete the instructions given to it
-            List<TransactionLog.Obsoletion> obsoletions = new ArrayList<>();
+            List<LogTransaction.Obsoletion> obsoletions = new ArrayList<>();
             accumulate = prepareForObsoletion(removed, txnLogs, obsoletions, accumulate);
             try
             {