private static void assertFiles(
      String dirPath, Set<String> expectedFiles, boolean excludeNonExistingFiles)
      throws IOException {
    LogTransaction.waitForDeletions();

    File dir = new File(dirPath).getCanonicalFile();
    File[] files = dir.listFiles();
    if (files != null) {
      for (File file : files) {
        if (file.isDirectory()) continue;

        String filePath = file.getPath();
        assertTrue(
            String.format("%s not in [%s]", filePath, expectedFiles),
            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());
  }
  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);
    }
  }
  public static void main(String args[]) throws IOException {
    Options options = Options.parseArgs(args);
    try {
      // load keyspace descriptions.
      DatabaseDescriptor.loadSchemas(false);

      if (Schema.instance.getCFMetaData(options.keyspace, options.cf) == null)
        throw new IllegalArgumentException(
            String.format("Unknown keyspace/columnFamily %s.%s", options.keyspace, options.cf));

      Keyspace keyspace = Keyspace.openWithoutSSTables(options.keyspace);
      ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(options.cf);

      OutputHandler handler = new OutputHandler.SystemOutput(false, options.debug);
      Directories.SSTableLister lister = cfs.directories.sstableLister();
      if (options.snapshot != null) lister.onlyBackups(true).snapshots(options.snapshot);
      else lister.includeBackups(false);

      Collection<SSTableReader> readers = new ArrayList<SSTableReader>();

      // Upgrade sstables
      for (Map.Entry<Descriptor, Set<Component>> entry : lister.list().entrySet()) {
        Set<Component> components = entry.getValue();
        if (!components.contains(Component.DATA) || !components.contains(Component.PRIMARY_INDEX))
          continue;

        try {
          SSTableReader sstable =
              SSTableReader.openNoValidation(entry.getKey(), components, cfs.metadata);
          if (sstable.descriptor.version.equals(Descriptor.Version.CURRENT)) continue;
          readers.add(sstable);
        } catch (Exception e) {
          JVMStabilityInspector.inspectThrowable(e);
          System.err.println(String.format("Error Loading %s: %s", entry.getKey(), e.getMessage()));
          if (options.debug) e.printStackTrace(System.err);

          continue;
        }
      }

      int numSSTables = readers.size();
      handler.output("Found " + numSSTables + " sstables that need upgrading.");

      for (SSTableReader sstable : readers) {
        try {
          Upgrader upgrader = new Upgrader(cfs, sstable, handler);
          upgrader.upgrade();

          if (!options.keepSource) {
            // Remove the sstable (it's been copied by upgrade)
            System.out.format("Deleting table %s.%n", sstable.descriptor.baseFilename());
            sstable.markObsolete();
            sstable.selfRef().release();
          }
        } catch (Exception e) {
          System.err.println(String.format("Error upgrading %s: %s", sstable, e.getMessage()));
          if (options.debug) e.printStackTrace(System.err);
        }
      }
      CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
      SSTableDeletingTask.waitForDeletions();
      System.exit(0);
    } 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());
    }
  }
  @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();
    }
  }