@Test
  public void testExportSuperCf() throws IOException {
    File tempSS = createTemporarySSTable("Keyspace1", "Super4");
    ColumnFamily cfamily = ColumnFamily.create("Keyspace1", "Super4");
    IPartitioner<?> partitioner = DatabaseDescriptor.getPartitioner();
    DataOutputBuffer dob = new DataOutputBuffer();
    SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, partitioner);

    // Add rowA
    cfamily.addColumn(
        new QueryPath("Super4", "superA".getBytes(), "colA".getBytes()),
        "valA".getBytes(),
        1,
        false);
    ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false);
    writer.append(partitioner.decorateKey("rowA"), dob);
    dob.reset();
    cfamily.clear();

    // Add rowB
    cfamily.addColumn(
        new QueryPath("Super4", "superB".getBytes(), "colB".getBytes()),
        "valB".getBytes(),
        1,
        false);
    ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false);
    writer.append(partitioner.decorateKey("rowB"), dob);
    dob.reset();
    cfamily.clear();

    // Add rowExclude
    cfamily.addColumn(
        new QueryPath("Super4", "superX".getBytes(), "colX".getBytes()),
        "valX".getBytes(),
        1,
        false);
    ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false);
    dob.reset();
    cfamily.clear();

    SSTableReader reader = writer.closeAndOpenReader();

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

    JSONObject json = (JSONObject) JSONValue.parse(new FileReader(tempJson));

    JSONObject rowA = (JSONObject) json.get("rowA");
    JSONObject superA =
        (JSONObject) rowA.get(cfamily.getComparator().getString("superA".getBytes()));
    JSONArray subColumns = (JSONArray) superA.get("subColumns");
    JSONArray colA = (JSONArray) subColumns.get(0);
    JSONObject rowExclude = (JSONObject) json.get("rowExclude");
    assert Arrays.equals(hexToBytes((String) colA.get(1)), "valA".getBytes());
    assert !(Boolean) colA.get(3);
    assert rowExclude == null;
  }
  @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;
  }
  @Test
  public void testEnumeratekeys() throws IOException {
    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, "colA".getBytes()), "valA".getBytes(), 1, false);
    ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false);
    writer.append(partitioner.decorateKey("rowA"), dob);
    dob.reset();
    cfamily.clear();

    // Add rowB
    cfamily.addColumn(
        new QueryPath("Standard1", null, "colB".getBytes()), "valB".getBytes(), 1, false);
    ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, true);
    writer.append(partitioner.decorateKey("rowB"), dob);
    dob.reset();
    cfamily.clear();

    writer.closeAndOpenReader();

    // Enumerate and verify
    File temp = File.createTempFile("Standard1", ".txt");
    SSTableExport.enumeratekeys(writer.getFilename(), new PrintStream(temp.getPath()));

    FileReader file = new FileReader(temp);
    char[] buf = new char[(int) temp.length()];
    file.read(buf);
    String output = new String(buf);

    String sep = System.getProperty("line.separator");
    assert output.equals("rowA" + sep + "rowB" + sep) : output;
  }
  // binary search is notoriously more difficult to get right than it looks; this is lifted from
  // Harmony's Collections implementation
  public int binarySearch(RowPosition key) {
    int low = 0, mid = keys.length, high = mid - 1, result = -1;

    while (low <= high) {
      mid = (low + high) >> 1;
      result = -partitioner.decorateKey(ByteBuffer.wrap(keys[mid])).compareTo(key);

      if (result > 0) {
        low = mid + 1;
      } else if (result == 0) {
        return mid;
      } else {
        high = mid - 1;
      }
    }

    return -mid - (result < 0 ? 1 : 2);
  }
 private void resolve(String key, byte[] buffer) {
   columnFamilies.put(partitioner.decorateKey(key), buffer);
   currentSize.addAndGet(buffer.length + key.length());
 }