@Test
  public void testAbortOnlyOld() throws Throwable {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
    File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);

    LogTransaction log = new LogTransaction(OperationType.COMPACTION);
    assertNotNull(log);

    LogTransaction.SSTableTidier tidier = log.obsoleted(sstable);
    assertNotNull(tidier);

    tidier.abort();
    log.abort();

    sstable.selfRef().release();

    assertFiles(dataFolder.getPath(), new HashSet<>(sstable.getAllFilePaths()));
  }
      protected Throwable doAbort(Throwable accumulate) {
        tidier.abort();
        LogTransaction.waitForDeletions();

        Throwable ret = txnLogs.abort(accumulate);

        sstableNew.selfRef().release();
        sstableOld.selfRef().release();
        return ret;
      }
  @Test
  public void testRemoveUnfinishedLeftovers_commit() throws Throwable {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
    File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    SSTableReader sstableOld = sstable(dataFolder, cfs, 0, 128);
    SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);

    // simulate tracking sstables with a committed transaction (new log file deleted)
    LogTransaction log = new LogTransaction(OperationType.COMPACTION);
    assertNotNull(log);

    log.trackNew(sstableNew);
    LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld);

    // Fake a commit
    log.txnFile().commit();

    Set<File> tmpFiles =
        sstableOld.getAllFilePaths().stream().map(File::new).collect(Collectors.toSet());

    sstableNew.selfRef().release();
    sstableOld.selfRef().release();

    Assert.assertEquals(tmpFiles, 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(dataFolder.getPath(), new HashSet<>(sstableNew.getAllFilePaths()));

    // complete the transaction to avoid LEAK errors
    tidier.run();
    assertNull(log.complete(null));
  }