@Test
  public void testStateJumpToLeft() throws UnknownHostException {
    StorageService ss = StorageService.instance;
    TokenMetadata tmd = ss.getTokenMetadata();
    tmd.clearUnsafe();
    IPartitioner partitioner = RandomPartitioner.instance;
    VersionedValue.VersionedValueFactory valueFactory =
        new VersionedValue.VersionedValueFactory(partitioner);

    ArrayList<Token> endpointTokens = new ArrayList<Token>();
    ArrayList<Token> keyTokens = new ArrayList<Token>();
    List<InetAddress> hosts = new ArrayList<InetAddress>();
    List<UUID> hostIds = new ArrayList<UUID>();

    // create a ring of 6 nodes
    Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 7);

    // node hosts.get(2) goes jumps to left
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.left(
            Collections.singleton(endpointTokens.get(2)), Gossiper.computeExpireTime()));

    assertFalse(tmd.isMember(hosts.get(2)));

    // node hosts.get(4) goes to bootstrap
    Gossiper.instance.injectApplicationState(
        hosts.get(3),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(1))));
    ss.onChange(
        hosts.get(3),
        ApplicationState.STATUS,
        valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(1))));

    assertFalse(tmd.isMember(hosts.get(3)));
    assertEquals(1, tmd.getBootstrapTokens().size());
    assertEquals(hosts.get(3), tmd.getBootstrapTokens().get(keyTokens.get(1)));

    // and then directly to 'left'
    Gossiper.instance.injectApplicationState(
        hosts.get(2),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(1))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.left(Collections.singleton(keyTokens.get(1)), Gossiper.computeExpireTime()));

    assertTrue(tmd.getBootstrapTokens().size() == 0);
    assertFalse(tmd.isMember(hosts.get(2)));
    assertFalse(tmd.isLeaving(hosts.get(2)));
  }
  @Test
  public void testRemoveHostId() throws InterruptedException {
    ReplicationSink rSink = new ReplicationSink();
    SinkManager.add(rSink);

    // start removal in background and send replication confirmations
    final AtomicBoolean success = new AtomicBoolean(false);
    Thread remover =
        new Thread() {
          public void run() {
            try {
              ss.removeNode(removalId.toString());
            } catch (Exception e) {
              System.err.println(e);
              e.printStackTrace();
              return;
            }
            success.set(true);
          }
        };
    remover.start();

    Thread.sleep(1000); // make sure removal is waiting for confirmation

    assertTrue(tmd.isLeaving(removalhost));
    assertEquals(1, tmd.getLeavingEndpoints().size());

    for (InetAddress host : hosts) {
      MessageOut msg =
          new MessageOut(
              host,
              MessagingService.Verb.REPLICATION_FINISHED,
              null,
              null,
              Collections.<String, byte[]>emptyMap());
      MessagingService.instance().sendRR(msg, FBUtilities.getBroadcastAddress());
    }

    remover.join();

    assertTrue(success.get());
    assertTrue(tmd.getLeavingEndpoints().isEmpty());
  }
  @Test
  public void testStateJumpToLeaving() throws UnknownHostException {
    StorageService ss = StorageService.instance;
    TokenMetadata tmd = ss.getTokenMetadata();
    tmd.clearUnsafe();
    IPartitioner partitioner = RandomPartitioner.instance;
    VersionedValue.VersionedValueFactory valueFactory =
        new VersionedValue.VersionedValueFactory(partitioner);

    ArrayList<Token> endpointTokens = new ArrayList<Token>();
    ArrayList<Token> keyTokens = new ArrayList<Token>();
    List<InetAddress> hosts = new ArrayList<InetAddress>();
    List<UUID> hostIds = new ArrayList<UUID>();

    // create a ring or 5 nodes
    Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 6);

    // node 2 leaves with _different_ token
    Gossiper.instance.injectApplicationState(
        hosts.get(2),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(0))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.leaving(Collections.singleton(keyTokens.get(0))));

    assertEquals(keyTokens.get(0), tmd.getToken(hosts.get(2)));
    assertTrue(tmd.isLeaving(hosts.get(2)));
    assertNull(tmd.getEndpoint(endpointTokens.get(2)));

    // go to boostrap
    Gossiper.instance.injectApplicationState(
        hosts.get(2),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(1))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(1))));

    assertFalse(tmd.isLeaving(hosts.get(2)));
    assertEquals(1, tmd.getBootstrapTokens().size());
    assertEquals(hosts.get(2), tmd.getBootstrapTokens().get(keyTokens.get(1)));

    // jump to leaving again
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.leaving(Collections.singleton(keyTokens.get(1))));

    assertEquals(hosts.get(2), tmd.getEndpoint(keyTokens.get(1)));
    assertTrue(tmd.isLeaving(hosts.get(2)));
    assertTrue(tmd.getBootstrapTokens().isEmpty());

    // go to state left
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.left(Collections.singleton(keyTokens.get(1)), Gossiper.computeExpireTime()));

    assertFalse(tmd.isMember(hosts.get(2)));
    assertFalse(tmd.isLeaving(hosts.get(2)));
  }
  @Test
  public void testStateJumpToNormal() throws UnknownHostException {
    StorageService ss = StorageService.instance;
    TokenMetadata tmd = ss.getTokenMetadata();
    tmd.clearUnsafe();
    IPartitioner partitioner = RandomPartitioner.instance;
    VersionedValue.VersionedValueFactory valueFactory =
        new VersionedValue.VersionedValueFactory(partitioner);

    ArrayList<Token> endpointTokens = new ArrayList<Token>();
    ArrayList<Token> keyTokens = new ArrayList<Token>();
    List<InetAddress> hosts = new ArrayList<InetAddress>();
    List<UUID> hostIds = new ArrayList<UUID>();

    // create a ring or 5 nodes
    Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 6);

    // node 2 leaves
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.leaving(Collections.singleton(endpointTokens.get(2))));

    assertTrue(tmd.isLeaving(hosts.get(2)));
    assertEquals(endpointTokens.get(2), tmd.getToken(hosts.get(2)));

    // back to normal
    Gossiper.instance.injectApplicationState(
        hosts.get(2),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(2))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.normal(Collections.singleton(keyTokens.get(2))));

    assertTrue(tmd.getLeavingEndpoints().isEmpty());
    assertEquals(keyTokens.get(2), tmd.getToken(hosts.get(2)));

    // node 3 goes through leave and left and then jumps to normal at its new token
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.leaving(Collections.singleton(keyTokens.get(2))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.left(Collections.singleton(keyTokens.get(2)), Gossiper.computeExpireTime()));
    Gossiper.instance.injectApplicationState(
        hosts.get(2),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(4))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.normal(Collections.singleton(keyTokens.get(4))));

    assertTrue(tmd.getBootstrapTokens().isEmpty());
    assertTrue(tmd.getLeavingEndpoints().isEmpty());
    assertEquals(keyTokens.get(4), tmd.getToken(hosts.get(2)));
  }
  /**
   * Test whether write endpoints is correct when the node is leaving. Uses StorageService.onChange
   * and does not manipulate token metadata directly.
   */
  @Test
  public void newTestWriteEndpointsDuringLeave() throws Exception {
    StorageService ss = StorageService.instance;
    final int RING_SIZE = 6;
    final int LEAVING_NODE = 3;

    TokenMetadata tmd = ss.getTokenMetadata();
    tmd.clearUnsafe();
    IPartitioner partitioner = RandomPartitioner.instance;
    VersionedValue.VersionedValueFactory valueFactory =
        new VersionedValue.VersionedValueFactory(partitioner);

    ArrayList<Token> endpointTokens = new ArrayList<Token>();
    ArrayList<Token> keyTokens = new ArrayList<Token>();
    List<InetAddress> hosts = new ArrayList<InetAddress>();
    List<UUID> hostIds = new ArrayList<UUID>();

    Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);

    Map<Token, List<InetAddress>> expectedEndpoints = new HashMap<Token, List<InetAddress>>();
    for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) {
      for (Token token : keyTokens) {
        List<InetAddress> endpoints = new ArrayList<InetAddress>();
        Iterator<Token> tokenIter = TokenMetadata.ringIterator(tmd.sortedTokens(), token, false);
        while (tokenIter.hasNext()) {
          endpoints.add(tmd.getEndpoint(tokenIter.next()));
        }
        expectedEndpoints.put(token, endpoints);
      }
    }

    // Third node leaves
    ss.onChange(
        hosts.get(LEAVING_NODE),
        ApplicationState.STATUS,
        valueFactory.leaving(Collections.singleton(endpointTokens.get(LEAVING_NODE))));
    assertTrue(tmd.isLeaving(hosts.get(LEAVING_NODE)));

    Thread.sleep(100); // because there is a tight race between submit and blockUntilFinished
    PendingRangeCalculatorService.instance.blockUntilFinished();

    AbstractReplicationStrategy strategy;
    for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces()) {
      strategy = getStrategy(keyspaceName, tmd);
      for (Token token : keyTokens) {
        int replicationFactor = strategy.getReplicationFactor();

        HashSet<InetAddress> actual =
            new HashSet<InetAddress>(
                tmd.getWriteEndpoints(
                    token,
                    keyspaceName,
                    strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
        HashSet<InetAddress> expected = new HashSet<InetAddress>();

        for (int i = 0; i < replicationFactor; i++) {
          expected.add(expectedEndpoints.get(token).get(i));
        }

        // if the leaving node is in the endpoint list,
        // then we should expect it plus one extra for when it's gone
        if (expected.contains(hosts.get(LEAVING_NODE)))
          expected.add(expectedEndpoints.get(token).get(replicationFactor));

        assertEquals("mismatched endpoint sets", expected, actual);
      }
    }
  }
  @Test
  public void testStateJumpToBootstrap() throws UnknownHostException {
    StorageService ss = StorageService.instance;
    TokenMetadata tmd = ss.getTokenMetadata();
    tmd.clearUnsafe();
    IPartitioner partitioner = RandomPartitioner.instance;
    VersionedValue.VersionedValueFactory valueFactory =
        new VersionedValue.VersionedValueFactory(partitioner);

    ArrayList<Token> endpointTokens = new ArrayList<Token>();
    ArrayList<Token> keyTokens = new ArrayList<Token>();
    List<InetAddress> hosts = new ArrayList<InetAddress>();
    List<UUID> hostIds = new ArrayList<UUID>();

    // create a ring or 5 nodes
    Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 7);

    // node 2 leaves
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.leaving(Collections.singleton(endpointTokens.get(2))));

    // don't bother to test pending ranges here, that is extensively tested by other
    // tests. Just check that the node is in appropriate lists.
    assertTrue(tmd.isMember(hosts.get(2)));
    assertTrue(tmd.isLeaving(hosts.get(2)));
    assertTrue(tmd.getBootstrapTokens().isEmpty());

    // Bootstrap the node immedidiately to keyTokens.get(4) without going through STATE_LEFT
    Gossiper.instance.injectApplicationState(
        hosts.get(2),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(4))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(4))));

    assertFalse(tmd.isMember(hosts.get(2)));
    assertFalse(tmd.isLeaving(hosts.get(2)));
    assertEquals(hosts.get(2), tmd.getBootstrapTokens().get(keyTokens.get(4)));

    // Bootstrap node hosts.get(3) to keyTokens.get(1)
    Gossiper.instance.injectApplicationState(
        hosts.get(3),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(1))));
    ss.onChange(
        hosts.get(3),
        ApplicationState.STATUS,
        valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(1))));

    assertFalse(tmd.isMember(hosts.get(3)));
    assertFalse(tmd.isLeaving(hosts.get(3)));
    assertEquals(hosts.get(2), tmd.getBootstrapTokens().get(keyTokens.get(4)));
    assertEquals(hosts.get(3), tmd.getBootstrapTokens().get(keyTokens.get(1)));

    // Bootstrap node hosts.get(2) further to keyTokens.get(3)
    Gossiper.instance.injectApplicationState(
        hosts.get(2),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(3))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(3))));

    assertFalse(tmd.isMember(hosts.get(2)));
    assertFalse(tmd.isLeaving(hosts.get(2)));
    assertEquals(hosts.get(2), tmd.getBootstrapTokens().get(keyTokens.get(3)));
    assertNull(tmd.getBootstrapTokens().get(keyTokens.get(4)));
    assertEquals(hosts.get(3), tmd.getBootstrapTokens().get(keyTokens.get(1)));

    // Go to normal again for both nodes
    Gossiper.instance.injectApplicationState(
        hosts.get(3),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(2))));
    Gossiper.instance.injectApplicationState(
        hosts.get(2),
        ApplicationState.TOKENS,
        valueFactory.tokens(Collections.singleton(keyTokens.get(3))));
    ss.onChange(
        hosts.get(2),
        ApplicationState.STATUS,
        valueFactory.normal(Collections.singleton(keyTokens.get(3))));
    ss.onChange(
        hosts.get(3),
        ApplicationState.STATUS,
        valueFactory.normal(Collections.singleton(keyTokens.get(2))));

    assertTrue(tmd.isMember(hosts.get(2)));
    assertFalse(tmd.isLeaving(hosts.get(2)));
    assertEquals(keyTokens.get(3), tmd.getToken(hosts.get(2)));
    assertTrue(tmd.isMember(hosts.get(3)));
    assertFalse(tmd.isLeaving(hosts.get(3)));
    assertEquals(keyTokens.get(2), tmd.getToken(hosts.get(3)));

    assertTrue(tmd.getBootstrapTokens().isEmpty());
  }