@Test
  public void testImportSimpleCf() throws IOException, ParseException {
    // Import JSON to temp SSTable file
    String jsonUrl = getClass().getClassLoader().getResource("SimpleCF.json").getPath();
    File tempSS = tempSSTableFile("Keyspace1", "Standard1");
    SSTableImport.importJson(jsonUrl, "Keyspace1", "Standard1", tempSS.getPath());

    // Verify results
    SSTableReader reader =
        SSTableReader.open(tempSS.getPath(), DatabaseDescriptor.getPartitioner());
    QueryFilter qf =
        QueryFilter.getNamesFilter(
            "rowA", new QueryPath("Standard1", null, null), "colAA".getBytes());
    ColumnFamily cf = qf.getSSTableColumnIterator(reader).getColumnFamily();
    assert Arrays.equals(cf.getColumn("colAA".getBytes()).value(), hexToBytes("76616c4141"));
  }
  @Test
  public void testRoundTripStandardCf() throws IOException, ParseException {
    File tempSS = createTemporarySSTable("Keyspace1", "Standard1");
    ColumnFamily cfamily = ColumnFamily.create("Keyspace1", "Standard1");
    IPartitioner<?> partitioner = DatabaseDescriptor.getPartitioner();
    DataOutputBuffer dob = new DataOutputBuffer();
    SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, partitioner);

    // Add rowA
    cfamily.addColumn(
        new QueryPath("Standard1", null, "name".getBytes()), "val".getBytes(), 1, false);
    ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false);
    writer.append(partitioner.decorateKey("rowA"), dob);
    dob.reset();
    cfamily.clear();

    // Add rowExclude
    cfamily.addColumn(
        new QueryPath("Standard1", null, "name".getBytes()), "val".getBytes(), 1, false);
    ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false);
    writer.append(partitioner.decorateKey("rowExclude"), dob);
    dob.reset();
    cfamily.clear();

    SSTableReader reader = writer.closeAndOpenReader();

    // Export to JSON and verify
    File tempJson = File.createTempFile("Standard1", ".json");
    SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[] {"rowExclude"});

    // Import JSON to another SSTable file
    File tempSS2 = createTemporarySSTable("Keyspace1", "Standard1");
    SSTableImport.importJson(tempJson.getPath(), "Keyspace1", "Standard1", tempSS2.getPath());

    reader = SSTableReader.open(tempSS2.getPath(), DatabaseDescriptor.getPartitioner());
    NamesQueryFilter qf =
        new NamesQueryFilter("rowA", new QueryPath("Standard1", null, null), "name".getBytes());
    ColumnFamily cf = qf.getSSTableColumnIterator(reader).getColumnFamily();
    assertTrue(cf != null);
    assertTrue(Arrays.equals(cf.getColumn("name".getBytes()).value(), hexToBytes("76616c")));

    qf =
        new NamesQueryFilter(
            "rowExclude", new QueryPath("Standard1", null, null), "name".getBytes());
    cf = qf.getSSTableColumnIterator(reader).getColumnFamily();
    assert cf == null;
  }