@SuppressWarnings("resource") // log and writer closed by SSTableTxnWriter
  public SSTableTxnWriter createFlushWriter(
      String filename, PartitionColumns columns, EncodingStats stats) {
    // we operate "offline" here, as we expose the resulting reader consciously when done
    // (although we may want to modify this behaviour in future, to encapsulate full flush behaviour
    // in LifecycleTransaction)
    LifecycleTransaction txn = null;
    try {
      txn = LifecycleTransaction.offline(OperationType.FLUSH);
      MetadataCollector sstableMetadataCollector =
          new MetadataCollector(cfs.metadata.comparator)
              .commitLogIntervals(
                  new IntervalSet(commitLogLowerBound.get(), commitLogUpperBound.get()));

      return new SSTableTxnWriter(
          txn,
          cfs.createSSTableMultiWriter(
              Descriptor.fromFilename(filename),
              (long) partitions.size(),
              ActiveRepairService.UNREPAIRED_SSTABLE,
              sstableMetadataCollector,
              new SerializationHeader(true, cfs.metadata, columns, stats),
              txn));
    } catch (Throwable t) {
      if (txn != null) txn.close();
      throw t;
    }
  }
  public void switchCompactionLocation(Directories.DataDirectory location) {
    @SuppressWarnings("resource")
    SSTableWriter writer =
        SSTableWriter.create(
            Descriptor.fromFilename(
                cfs.getSSTablePath(getDirectories().getLocationForDisk(location))),
            estimatedTotalKeys / estimatedSSTables,
            minRepairedAt,
            cfs.metadata,
            new MetadataCollector(allSSTables, cfs.metadata.comparator, level),
            SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
            txn);

    sstableWriter.switchWriter(writer);
  }
  @Test
  public void testGetNewNames() throws IOException {
    Descriptor desc =
        Descriptor.fromFilename(
            new File("Keyspace1", "Keyspace1-Standard1-ia-500-Data.db").toString());
    // assert !desc.isLatestVersion; // minimum compatible version -- for now it is the latest as
    // well
    PendingFile inContext =
        new PendingFile(
            null, desc, "Data.db", Arrays.asList(Pair.create(0L, 1L)), OperationType.BOOTSTRAP);

    PendingFile outContext = StreamIn.getContextMapping(inContext);
    // filename and generation are expected to have changed
    assert !inContext.getFilename().equals(outContext.getFilename());

    // nothing else should
    assertEquals(inContext.component, outContext.component);
    assertEquals(desc.ksname, outContext.desc.ksname);
    assertEquals(desc.cfname, outContext.desc.cfname);
    assertEquals(desc.version, outContext.desc.version);
  }
  /**
   * Given arguments specifying an SSTable, and optionally an output file, export the contents of
   * the SSTable to JSON.
   *
   * @param args command lines arguments
   * @throws ConfigurationException on configuration failure (wrong params given)
   */
  public static void main(String[] args) throws ConfigurationException {
    CommandLineParser parser = new PosixParser();
    try {
      cmd = parser.parse(options, args);
    } catch (ParseException e1) {
      System.err.println(e1.getMessage());
      printUsage();
      System.exit(1);
    }

    if (cmd.getArgs().length != 1) {
      System.err.println("You must supply exactly one sstable");
      printUsage();
      System.exit(1);
    }

    String[] keys = cmd.getOptionValues(KEY_OPTION);
    HashSet<String> excludes =
        new HashSet<>(
            Arrays.asList(
                cmd.getOptionValues(EXCLUDE_KEY_OPTION) == null
                    ? new String[0]
                    : cmd.getOptionValues(EXCLUDE_KEY_OPTION)));
    String ssTableFileName = new File(cmd.getArgs()[0]).getAbsolutePath();

    if (Descriptor.isLegacyFile(new File(ssTableFileName))) {
      System.err.println("Unsupported legacy sstable");
      System.exit(1);
    }
    if (!new File(ssTableFileName).exists()) {
      System.err.println("Cannot find file " + ssTableFileName);
      System.exit(1);
    }
    Descriptor desc = Descriptor.fromFilename(ssTableFileName);
    try {
      CFMetaData metadata = metadataFromSSTable(desc);
      if (cmd.hasOption(ENUMERATE_KEYS_OPTION)) {
        JsonTransformer.keysToJson(
            null,
            iterToStream(new KeyIterator(desc, metadata)),
            cmd.hasOption(RAW_TIMESTAMPS),
            metadata,
            System.out);
      } else {
        SSTableReader sstable = SSTableReader.openNoValidation(desc, metadata);
        IPartitioner partitioner = sstable.getPartitioner();
        final ISSTableScanner currentScanner;
        if ((keys != null) && (keys.length > 0)) {
          List<AbstractBounds<PartitionPosition>> bounds =
              Arrays.stream(keys)
                  .filter(key -> !excludes.contains(key))
                  .map(metadata.getKeyValidator()::fromString)
                  .map(partitioner::decorateKey)
                  .sorted()
                  .map(DecoratedKey::getToken)
                  .map(token -> new Bounds<>(token.minKeyBound(), token.maxKeyBound()))
                  .collect(Collectors.toList());
          currentScanner = sstable.getScanner(bounds.iterator());
        } else {
          currentScanner = sstable.getScanner();
        }
        Stream<UnfilteredRowIterator> partitions =
            iterToStream(currentScanner)
                .filter(
                    i ->
                        excludes.isEmpty()
                            || !excludes.contains(
                                metadata.getKeyValidator().getString(i.partitionKey().getKey())));
        if (cmd.hasOption(DEBUG_OUTPUT_OPTION)) {
          AtomicLong position = new AtomicLong();
          partitions.forEach(
              partition -> {
                position.set(currentScanner.getCurrentPosition());

                if (!partition.partitionLevelDeletion().isLive()) {
                  System.out.println(
                      "["
                          + metadata.getKeyValidator().getString(partition.partitionKey().getKey())
                          + "]@"
                          + position.get()
                          + " "
                          + partition.partitionLevelDeletion());
                }
                if (!partition.staticRow().isEmpty()) {
                  System.out.println(
                      "["
                          + metadata.getKeyValidator().getString(partition.partitionKey().getKey())
                          + "]@"
                          + position.get()
                          + " "
                          + partition.staticRow().toString(metadata, true));
                }
                partition.forEachRemaining(
                    row -> {
                      System.out.println(
                          "["
                              + metadata
                                  .getKeyValidator()
                                  .getString(partition.partitionKey().getKey())
                              + "]@"
                              + position.get()
                              + " "
                              + row.toString(metadata, false, true));
                      position.set(currentScanner.getCurrentPosition());
                    });
              });
        } else {
          JsonTransformer.toJson(
              currentScanner, partitions, cmd.hasOption(RAW_TIMESTAMPS), metadata, System.out);
        }
      }
    } catch (IOException e) {
      // throwing exception outside main with broken pipe causes windows cmd to hang
      e.printStackTrace(System.err);
    }

    System.exit(0);
  }