public Map<Token, Float> describeOwnership(List<Token> sortedTokens) {
      // allTokens will contain the count and be returned, sorted_ranges is shorthand for
      // token<->token math.
      Map<Token, Float> allTokens = new HashMap<Token, Float>();
      List<Range<Token>> sortedRanges = new ArrayList<Range<Token>>();

      // this initializes the counts to 0 and calcs the ranges in order.
      Token lastToken = sortedTokens.get(sortedTokens.size() - 1);
      for (Token node : sortedTokens) {
        allTokens.put(node, new Float(0.0));
        sortedRanges.add(new Range<Token>(lastToken, node));
        lastToken = node;
      }

      for (String ks : Schema.instance.getKeyspaces()) {
        for (CFMetaData cfmd : Schema.instance.getKSMetaData(ks).cfMetaData().values()) {
          for (Range<Token> r : sortedRanges) {
            // Looping over every KS:CF:Range, get the splits size and add it to the count
            allTokens.put(
                r.right,
                allTokens.get(r.right)
                    + StorageService.instance.getSplits(ks, cfmd.cfName, r, 1).size());
          }
        }
      }

      // Sum every count up and divide count/total for the fractional ownership.
      Float total = new Float(0.0);
      for (Float f : allTokens.values()) total += f;
      for (Map.Entry<Token, Float> row : allTokens.entrySet())
        allTokens.put(row.getKey(), row.getValue() / total);

      return allTokens;
    }
  @Test
  public void testGetSliceWithCollision() throws Exception {
    Keyspace keyspace = Keyspace.open(KEYSPACE);
    ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
    cfs.clearUnsafe();

    insert("k1", "k2", "k3"); // token = 2
    insert("key1", "key2", "key3"); // token = 4
    insert("longKey1", "longKey2"); // token = 8

    List<Row> rows =
        cfs.getRangeSlice(
            new Bounds<RowPosition>(dk("k2"), dk("key2")), null, new IdentityQueryFilter(), 10000);
    assert rows.size() == 4 : "Expecting 4 keys, got " + rows.size();
    assert rows.get(0).key.key.equals(ByteBufferUtil.bytes("k2"));
    assert rows.get(1).key.key.equals(ByteBufferUtil.bytes("k3"));
    assert rows.get(2).key.key.equals(ByteBufferUtil.bytes("key1"));
    assert rows.get(3).key.key.equals(ByteBufferUtil.bytes("key2"));
  }
  /**
   * Writes out a bunch of mutations for a single column family.
   *
   * @param mutations A group of Mutations for the same keyspace and column family.
   * @return The ColumnFamilyStore that was used.
   */
  public static ColumnFamilyStore writeColumnFamily(List<Mutation> mutations) {
    IMutation first = mutations.get(0);
    String keyspaceName = first.getKeyspaceName();
    UUID cfid = first.getColumnFamilyIds().iterator().next();

    for (Mutation rm : mutations) rm.applyUnsafe();

    ColumnFamilyStore store = Keyspace.open(keyspaceName).getColumnFamilyStore(cfid);
    store.forceBlockingFlush();
    return store;
  }
 public HIterator(int... tokens) {
   List<Token> tlist = new LinkedList<Token>();
   for (int token : tokens) tlist.add(tok(token));
   this.tokens = tlist.iterator();
 }
 public static Future<?> compactAll(ColumnFamilyStore cfs, int gcBefore) {
   List<Descriptor> descriptors = new ArrayList<>();
   for (SSTableReader sstable : cfs.getSSTables()) descriptors.add(sstable.descriptor);
   return CompactionManager.instance.submitUserDefined(cfs, descriptors, gcBefore);
 }
  /** Creates initial set of nodes and tokens. Nodes are added to StorageService as 'normal' */
  public static void createInitialRing(
      StorageService ss,
      IPartitioner partitioner,
      List<Token> endpointTokens,
      List<Token> keyTokens,
      List<InetAddress> hosts,
      List<UUID> hostIds,
      int howMany)
      throws UnknownHostException {
    // Expand pool of host IDs as necessary
    for (int i = hostIdPool.size(); i < howMany; i++) hostIdPool.add(UUID.randomUUID());

    for (int i = 0; i < howMany; i++) {
      endpointTokens.add(new BigIntegerToken(String.valueOf(10 * i)));
      keyTokens.add(new BigIntegerToken(String.valueOf(10 * i + 5)));
      hostIds.add(hostIdPool.get(i));
    }

    for (int i = 0; i < endpointTokens.size(); i++) {
      InetAddress ep = InetAddress.getByName("127.0.0." + String.valueOf(i + 1));
      Gossiper.instance.initializeNodeUnsafe(ep, hostIds.get(i), 1);
      Gossiper.instance.injectApplicationState(
          ep,
          ApplicationState.TOKENS,
          new VersionedValue.VersionedValueFactory(partitioner)
              .tokens(Collections.singleton(endpointTokens.get(i))));
      ss.onChange(
          ep,
          ApplicationState.STATUS,
          new VersionedValue.VersionedValueFactory(partitioner)
              .normal(Collections.singleton(endpointTokens.get(i))));
      hosts.add(ep);
    }

    // check that all nodes are in token metadata
    for (int i = 0; i < endpointTokens.size(); ++i)
      assertTrue(ss.getTokenMetadata().isMember(hosts.get(i)));
  }