/**
   * Update token map with a set of token/endpoint pairs in normal state.
   *
   * <p>Prefer this whenever there are multiple pairs to update, as each update (whether a single or
   * multiple) is expensive (CASSANDRA-3831).
   *
   * @param endpointTokens
   */
  public void updateNormalTokens(Multimap<InetAddress, Token> endpointTokens) {
    if (endpointTokens.isEmpty()) return;

    lock.writeLock().lock();
    try {
      boolean shouldSortTokens = false;
      for (InetAddress endpoint : endpointTokens.keySet()) {
        Collection<Token> tokens = endpointTokens.get(endpoint);

        assert tokens != null && !tokens.isEmpty();

        bootstrapTokens.removeValue(endpoint);
        tokenToEndpointMap.removeValue(endpoint);
        topology.addEndpoint(endpoint);
        leavingEndpoints.remove(endpoint);
        removeFromMoving(endpoint); // also removing this endpoint from moving

        for (Token token : tokens) {
          InetAddress prev = tokenToEndpointMap.put(token, endpoint);
          if (!endpoint.equals(prev)) {
            if (prev != null)
              logger.warn("Token {} changing ownership from {} to {}", token, prev, endpoint);
            shouldSortTokens = true;
          }
        }
      }

      if (shouldSortTokens) sortedTokens = sortTokens();
    } finally {
      lock.writeLock().unlock();
    }
  }
 /**
  * @return a (stable copy, won't be modified) Token to Endpoint map for all the normal and
  *     bootstrapping nodes in the cluster.
  */
 public Map<Token, InetAddress> getNormalAndBootstrappingTokenToEndpointMap() {
   lock.readLock().lock();
   try {
     Map<Token, InetAddress> map =
         new HashMap<Token, InetAddress>(tokenToEndpointMap.size() + bootstrapTokens.size());
     map.putAll(tokenToEndpointMap);
     map.putAll(bootstrapTokens);
     return map;
   } finally {
     lock.readLock().unlock();
   }
 }
 /** used by tests */
 public void clearUnsafe() {
   lock.writeLock().lock();
   try {
     bootstrapTokens.clear();
     tokenToEndpointMap.clear();
     topology.clear();
     leavingEndpoints.clear();
     pendingRanges.clear();
     endpointToHostIdMap.clear();
     invalidateCachedRings();
   } finally {
     lock.writeLock().unlock();
   }
 }
 public Token getSuccessor(Token token) {
   List tokens = sortedTokens();
   int index = Collections.binarySearch(tokens, token);
   assert index >= 0
       : token + " not found in " + StringUtils.join(tokenToEndpointMap.keySet(), ", ");
   return (Token) ((index == (tokens.size() - 1)) ? tokens.get(0) : tokens.get(index + 1));
 }
  public void removeEndpoint(InetAddress endpoint) {
    assert endpoint != null;

    lock.writeLock().lock();
    try {
      bootstrapTokens.removeValue(endpoint);
      tokenToEndpointMap.removeValue(endpoint);
      topology.removeEndpoint(endpoint);
      leavingEndpoints.remove(endpoint);
      endpointToHostIdMap.remove(endpoint);
      sortedTokens = sortTokens();
      invalidateCachedRings();
    } finally {
      lock.writeLock().unlock();
    }
  }
  public String toString() {
    StringBuilder sb = new StringBuilder();
    lock.readLock().lock();
    try {
      Set<InetAddress> eps = tokenToEndpointMap.inverse().keySet();

      if (!eps.isEmpty()) {
        sb.append("Normal Tokens:");
        sb.append(System.getProperty("line.separator"));
        for (InetAddress ep : eps) {
          sb.append(ep);
          sb.append(":");
          sb.append(tokenToEndpointMap.inverse().get(ep));
          sb.append(System.getProperty("line.separator"));
        }
      }

      if (!bootstrapTokens.isEmpty()) {
        sb.append("Bootstrapping Tokens:");
        sb.append(System.getProperty("line.separator"));
        for (Map.Entry<Token, InetAddress> entry : bootstrapTokens.entrySet()) {
          sb.append(entry.getValue()).append(":").append(entry.getKey());
          sb.append(System.getProperty("line.separator"));
        }
      }

      if (!leavingEndpoints.isEmpty()) {
        sb.append("Leaving Endpoints:");
        sb.append(System.getProperty("line.separator"));
        for (InetAddress ep : leavingEndpoints) {
          sb.append(ep);
          sb.append(System.getProperty("line.separator"));
        }
      }

      if (!pendingRanges.isEmpty()) {
        sb.append("Pending Ranges:");
        sb.append(System.getProperty("line.separator"));
        sb.append(printPendingRanges());
      }
    } finally {
      lock.readLock().unlock();
    }

    return sb.toString();
  }
 public InetAddress getEndpoint(Token token) {
   lock.readLock().lock();
   try {
     return tokenToEndpointMap.get(token);
   } finally {
     lock.readLock().unlock();
   }
 }
  public boolean isMember(InetAddress endpoint) {
    assert endpoint != null;

    lock.readLock().lock();
    try {
      return tokenToEndpointMap.inverse().containsKey(endpoint);
    } finally {
      lock.readLock().unlock();
    }
  }
  public void removeBootstrapTokens(Collection<Token> tokens) {
    assert tokens != null && !tokens.isEmpty();

    lock.writeLock().lock();
    try {
      for (Token token : tokens) bootstrapTokens.remove(token);
    } finally {
      lock.writeLock().unlock();
    }
  }
 /** @return an endpoint to token multimap representation of tokenToEndpointMap (a copy) */
 public Multimap<InetAddress, Token> getEndpointToTokenMapForReading() {
   lock.readLock().lock();
   try {
     Multimap<InetAddress, Token> cloned = HashMultimap.create();
     for (Map.Entry<Token, InetAddress> entry : tokenToEndpointMap.entrySet())
       cloned.put(entry.getValue(), entry.getKey());
     return cloned;
   } finally {
     lock.readLock().unlock();
   }
 }
  public Collection<Token> getTokens(InetAddress endpoint) {
    assert endpoint != null;
    assert isMember(endpoint); // don't want to return nulls

    lock.readLock().lock();
    try {
      return new ArrayList<Token>(tokenToEndpointMap.inverse().get(endpoint));
    } finally {
      lock.readLock().unlock();
    }
  }
 /** @return the number of nodes bootstrapping into source's primary range */
 public int pendingRangeChanges(InetAddress source) {
   int n = 0;
   Collection<Range<Token>> sourceRanges = getPrimaryRangesFor(getTokens(source));
   lock.readLock().lock();
   try {
     for (Token token : bootstrapTokens.keySet())
       for (Range<Token> range : sourceRanges) if (range.contains(token)) n++;
   } finally {
     lock.readLock().unlock();
   }
   return n;
 }
  public void addBootstrapTokens(Collection<Token> tokens, InetAddress endpoint) {
    assert tokens != null && !tokens.isEmpty();
    assert endpoint != null;

    lock.writeLock().lock();
    try {

      InetAddress oldEndpoint;

      for (Token token : tokens) {
        oldEndpoint = bootstrapTokens.get(token);
        if (oldEndpoint != null && !oldEndpoint.equals(endpoint))
          throw new RuntimeException(
              "Bootstrap Token collision between "
                  + oldEndpoint
                  + " and "
                  + endpoint
                  + " (token "
                  + token);

        oldEndpoint = tokenToEndpointMap.get(token);
        if (oldEndpoint != null && !oldEndpoint.equals(endpoint))
          throw new RuntimeException(
              "Bootstrap Token collision between "
                  + oldEndpoint
                  + " and "
                  + endpoint
                  + " (token "
                  + token);
      }

      bootstrapTokens.removeValue(endpoint);

      for (Token token : tokens) bootstrapTokens.put(token, endpoint);
    } finally {
      lock.writeLock().unlock();
    }
  }
 private ArrayList<Token> sortTokens() {
   return new ArrayList<Token>(tokenToEndpointMap.keySet());
 }