@Test
  public void testRemoveUnfinishedLeftovers_abort_multipleFolders() throws Throwable {
    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)
    };

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

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

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

    Collection<File> logFiles = log.logFiles();
    Assert.assertEquals(2, logFiles.size());

    // fake an abort
    log.txnFile().abort();

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

    // test listing
    Assert.assertEquals(
        sstables[1].getAllFilePaths().stream().map(File::new).collect(Collectors.toSet()),
        getTemporaryFiles(dataFolder1));
    Assert.assertEquals(
        sstables[3].getAllFilePaths().stream().map(File::new).collect(Collectors.toSet()),
        getTemporaryFiles(dataFolder2));

    // normally called at startup
    LogTransaction.removeUnfinishedLeftovers(Arrays.asList(dataFolder1, dataFolder2));

    // old tables should be only table left
    assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[0].getAllFilePaths()));
    assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[2].getAllFilePaths()));

    // complete the transaction to avoid LEAK errors
    Arrays.stream(tidiers).forEach(LogTransaction.SSTableTidier::run);
    assertNull(log.complete(null));
  }
    public static Options parseArgs(String cmdArgs[]) {
      CommandLineParser parser = new GnuParser();
      CmdLineOptions options = getCmdLineOptions();
      try {
        CommandLine cmd = parser.parse(options, cmdArgs, false);

        if (cmd.hasOption(HELP_OPTION)) {
          printUsage(options);
          System.exit(0);
        }

        String[] args = cmd.getArgs();
        if (args.length == 0) {
          System.err.println("No sstables to split");
          printUsage(options);
          System.exit(1);
        }
        Options opts = new Options(Arrays.asList(args));
        opts.debug = cmd.hasOption(DEBUG_OPTION);
        opts.verbose = cmd.hasOption(VERBOSE_OPTION);
        opts.snapshot = !cmd.hasOption(NO_SNAPSHOT_OPTION);
        opts.sizeInMB = DEFAULT_SSTABLE_SIZE;

        if (cmd.hasOption(SIZE_OPTION))
          opts.sizeInMB = Integer.valueOf(cmd.getOptionValue(SIZE_OPTION));

        return opts;
      } catch (ParseException e) {
        errorMsg(e.getMessage(), options);
        return null;
      }
    }
  @Test
  public void testAbortMultipleFolders() throws Throwable {
    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)
    };

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

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

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

    Arrays.stream(tidiers).forEach(LogTransaction.SSTableTidier::abort);
    log.abort();

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

    assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[0].getAllFilePaths()));
    assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[2].getAllFilePaths()));
  }
 static Set<File> listFiles(File folder, Directories.FileType... types) {
   Collection<Directories.FileType> match = Arrays.asList(types);
   return new LogAwareFileLister(
           folder.toPath(), (file, type) -> match.contains(type), Directories.OnTxnErr.IGNORE)
       .list()
       .stream()
       .map(
           f -> {
             try {
               return f.getCanonicalFile();
             } catch (IOException e) {
               throw new IOError(e);
             }
           })
       .collect(Collectors.toSet());
 }
  public static void main(String args[]) throws IOException {
    Options options = Options.parseArgs(args);
    try {
      // load keyspace descriptions.
      DatabaseDescriptor.loadSchemas();

      String ksName = null;
      String cfName = null;
      Map<Descriptor, Set<Component>> parsedFilenames = new HashMap<Descriptor, Set<Component>>();
      for (String filename : options.filenames) {
        File file = new File(filename);
        if (!file.exists()) {
          System.out.println("Skipping inexisting file " + file);
          continue;
        }

        Pair<Descriptor, Component> pair =
            SSTable.tryComponentFromFilename(file.getParentFile(), file.getName());
        if (pair == null) {
          System.out.println("Skipping non sstable file " + file);
          continue;
        }
        Descriptor desc = pair.left;

        if (ksName == null) ksName = desc.ksname;
        else if (!ksName.equals(desc.ksname))
          throw new IllegalArgumentException("All sstables must be part of the same keyspace");

        if (cfName == null) cfName = desc.cfname;
        else if (!cfName.equals(desc.cfname))
          throw new IllegalArgumentException("All sstables must be part of the same column family");

        Set<Component> components =
            new HashSet<Component>(
                Arrays.asList(
                    new Component[] {
                      Component.DATA,
                      Component.PRIMARY_INDEX,
                      Component.FILTER,
                      Component.COMPRESSION_INFO,
                      Component.STATS
                    }));

        Iterator<Component> iter = components.iterator();
        while (iter.hasNext()) {
          Component component = iter.next();
          if (!(new File(desc.filenameFor(component)).exists())) iter.remove();
        }
        parsedFilenames.put(desc, components);
      }

      if (ksName == null || cfName == null) {
        System.err.println("No valid sstables to split");
        System.exit(1);
      }

      // Do not load sstables since they might be broken
      Table table = Table.openWithoutSSTables(ksName);
      ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);

      String snapshotName = "pre-split-" + System.currentTimeMillis();

      List<SSTableReader> sstables = new ArrayList<SSTableReader>();
      for (Map.Entry<Descriptor, Set<Component>> fn : parsedFilenames.entrySet()) {
        try {
          SSTableReader sstable =
              SSTableReader.openNoValidation(fn.getKey(), fn.getValue(), cfs.metadata);
          sstables.add(sstable);

          if (options.snapshot) {
            File snapshotDirectory =
                Directories.getSnapshotDirectory(sstable.descriptor, snapshotName);
            sstable.createLinks(snapshotDirectory.getPath());
          }

        } catch (Exception e) {
          System.err.println(String.format("Error Loading %s: %s", fn.getKey(), e.getMessage()));
          if (options.debug) e.printStackTrace(System.err);
        }
      }
      if (options.snapshot)
        System.out.println(
            String.format("Pre-split sstables snapshotted into snapshot %s", snapshotName));

      cfs.getDataTracker().markCompacting(sstables);
      for (SSTableReader sstable : sstables) {
        try {
          new SSTableSplitter(cfs, sstable, options.sizeInMB).split();

          // Remove the sstable
          sstable.markCompacted();
          sstable.releaseReference();
        } catch (Exception e) {
          System.err.println(String.format("Error splitting %s: %s", sstable, e.getMessage()));
          if (options.debug) e.printStackTrace(System.err);
        }
      }
      SSTableDeletingTask.waitForDeletions();
      System.exit(0); // We need that to stop non daemonized threads
    } catch (Exception e) {
      System.err.println(e.getMessage());
      if (options.debug) e.printStackTrace(System.err);
      System.exit(1);
    }
  }
  @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());
    }
  }
  private static void testRemoveUnfinishedLeftovers_multipleFolders_errorConditions(
      Consumer<LogTransaction> modifier, boolean shouldCommit) throws Throwable {
    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)
    };

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

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

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

    // fake some error condition on the txn logs
    modifier.accept(log);

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

    LogTransaction.removeUnfinishedLeftovers(Arrays.asList(dataFolder1, dataFolder2));
    LogTransaction.waitForDeletions();

    if (shouldCommit) {
      // only new sstables should still be there
      assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[1].getAllFilePaths()));
      assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[3].getAllFilePaths()));
    } else {
      // all files should still be there
      assertFiles(
          dataFolder1.getPath(),
          Sets.newHashSet(
              Iterables.concat(
                  sstables[0].getAllFilePaths(),
                  sstables[1].getAllFilePaths(),
                  Collections.singleton(log.logFilePaths().get(0)))));
      assertFiles(
          dataFolder2.getPath(),
          Sets.newHashSet(
              Iterables.concat(
                  sstables[2].getAllFilePaths(),
                  sstables[3].getAllFilePaths(),
                  Collections.singleton(log.logFilePaths().get(1)))));
    }

    // complete the transaction to avoid LEAK errors
    Arrays.stream(tidiers).forEach(LogTransaction.SSTableTidier::run);
    log.txnFile().commit(); // just anything to make sure transaction tidier will finish
    assertNull(log.complete(null));
  }