Skip to content

Commit

Permalink
revert unintentional commit of SS and AES
Browse files Browse the repository at this point in the history
git-svn-id: https://svn.apache.org/repos/asf/cassandra/branches/cassandra-0.6@930939 13f79535-47bb-0310-9956-ffa450edef68
  • Loading branch information
jbellis committed Apr 5, 2010
1 parent 09940ac commit ba63bfe
Show file tree
Hide file tree
Showing 2 changed files with 8 additions and 12 deletions.
17 changes: 7 additions & 10 deletions src/java/org/apache/cassandra/service/AntiEntropyService.java
Expand Up @@ -43,6 +43,9 @@

import org.apache.log4j.Logger;

import com.google.common.collect.Collections2;
import com.google.common.base.Predicates;

/**
* AntiEntropyService encapsulates "validating" (hashing) individual column families,
* exchanging MerkleTrees with remote nodes via a TreeRequest/Response conversation,
Expand Down Expand Up @@ -138,18 +141,12 @@ private ExpiringMap<InetAddress, TreePair> rendezvousPairs(CFPair cf)
/**
* Return all of the neighbors with whom we share data.
*/
public static Set<InetAddress> getNeighbors(String table)
private static Collection<InetAddress> getNeighbors(String table)
{
InetAddress local = FBUtilities.getLocalAddress();
StorageService ss = StorageService.instance;
Set<InetAddress> neighbors = new HashSet<InetAddress>();
Map<Range, List<InetAddress>> replicaSets = ss.getRangeToAddressMap(table);
for (Range range : ss.getLocalRanges(table))
{
// for every range stored locally (replica or original) collect neighbors storing copies
neighbors.addAll(replicaSets.get(range));
}
neighbors.remove(FBUtilities.getLocalAddress());
return neighbors;
return Collections2.filter(ss.getNaturalEndpoints(table, ss.getLocalToken()),
Predicates.not(Predicates.equalTo(local)));
}

/**
Expand Down
3 changes: 1 addition & 2 deletions src/java/org/apache/cassandra/service/StorageService.java
Expand Up @@ -1063,8 +1063,7 @@ public void forceTableRepair(final String tableName, final String... columnFamil
{
// request that all relevant endpoints generate trees
final MessagingService ms = MessagingService.instance;
final Set<InetAddress> endpoints = AntiEntropyService.getNeighbors(tableName);
endpoints.add(FBUtilities.getLocalAddress());
final List<InetAddress> endpoints = getNaturalEndpoints(tableName, getLocalToken());
for (ColumnFamilyStore cfStore : getValidColumnFamilies(tableName, columnFamilies))
{
Message request = TreeRequestVerbHandler.makeVerb(tableName, cfStore.getColumnFamilyName());
Expand Down

0 comments on commit ba63bfe

Please sign in to comment.