@Test
  public void testGetTemporaryFilesMultipleFolders() throws IOException {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);

    File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    File dataFolder1 = new File(origiFolder, "1");
    File dataFolder2 = new File(origiFolder, "2");
    Files.createDirectories(dataFolder1.toPath());
    Files.createDirectories(dataFolder2.toPath());

    SSTableReader[] sstables = {
      sstable(dataFolder1, cfs, 0, 128),
      sstable(dataFolder1, cfs, 1, 128),
      sstable(dataFolder2, cfs, 2, 128),
      sstable(dataFolder2, cfs, 3, 128)
    };

    // they should all have the same number of files since they are created in the same way
    int numSStableFiles = sstables[0].getAllFilePaths().size();

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

    for (File dataFolder : new File[] {dataFolder1, dataFolder2}) {
      Set<File> tmpFiles = getTemporaryFiles(dataFolder);
      assertNotNull(tmpFiles);
      assertEquals(0, tmpFiles.size());
    }

    LogTransaction.SSTableTidier[] tidiers = {
      log.obsoleted(sstables[0]), log.obsoleted(sstables[2])
    };

    log.trackNew(sstables[1]);
    log.trackNew(sstables[3]);

    for (File dataFolder : new File[] {dataFolder1, dataFolder2}) {
      Set<File> tmpFiles = getTemporaryFiles(dataFolder);
      assertNotNull(tmpFiles);
      assertEquals(numSStableFiles, tmpFiles.size());
    }

    log.finish();

    for (File dataFolder : new File[] {dataFolder1, dataFolder2}) {
      Set<File> tmpFiles = getTemporaryFiles(dataFolder);
      assertNotNull(tmpFiles);
      assertEquals(numSStableFiles, tmpFiles.size());
    }

    sstables[0].markObsolete(tidiers[0]);
    sstables[2].markObsolete(tidiers[1]);

    Arrays.stream(sstables).forEach(s -> s.selfRef().release());
    LogTransaction.waitForDeletions();

    for (File dataFolder : new File[] {dataFolder1, dataFolder2}) {
      Set<File> tmpFiles = getTemporaryFiles(dataFolder);
      assertNotNull(tmpFiles);
      assertEquals(0, tmpFiles.size());
    }
  }
  @Test
  public void testGetTemporaryFiles() throws IOException {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
    File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    SSTableReader sstable1 = sstable(dataFolder, cfs, 0, 128);

    Set<File> tmpFiles = getTemporaryFiles(dataFolder);
    assertNotNull(tmpFiles);
    assertEquals(0, tmpFiles.size());

    try (LogTransaction log = new LogTransaction(OperationType.WRITE)) {
      Directories directories = new Directories(cfs.metadata);

      File[] beforeSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());

      SSTableReader sstable2 = sstable(dataFolder, 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 = 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 = 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();
    }
  }