You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by ifesdjeen <gi...@git.apache.org> on 2018/09/05 17:49:41 UTC

[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

GitHub user ifesdjeen opened a pull request:

    https://github.com/apache/cassandra/pull/257

    Allow transient range owner to serve as repair coordinator

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ifesdjeen/cassandra tr-followup-1

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/cassandra/pull/257.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #257
    
----
commit aadc3c6dcd65ef915c1743cb133f1065ee2618c3
Author: Alex Petrov <ol...@...>
Date:   2018-09-04T17:38:27Z

    Imrpve DiskBoundaryManager, bring it into consistency with the rest of classes

commit c1a73a5eb799ee4d12a580af848d72eda21fd7b2
Author: Alex Petrov <ol...@...>
Date:   2018-09-04T17:39:19Z

    Improve repair scheduling

commit e47019f838f69694d943220e3fa2f42839dd8019
Author: Alex Petrov <ol...@...>
Date:   2018-09-04T17:40:08Z

    Avoid filtering already

commit 1d19e26fbbb97d76590d8b164feb61d185d74ba2
Author: Alex Petrov <ol...@...>
Date:   2018-09-05T08:16:51Z

    enable dtests

commit 3e61a44a6e4bc4ba0dd32b588905b01bb64aff7f
Author: Alex Petrov <al...@...>
Date:   2018-09-05T11:11:51Z

    Allow transient range owner to be repair coordinator

commit c9d4e02aeb9f527e4584721881de2794085c594f
Author: Alex Petrov <al...@...>
Date:   2018-09-05T13:14:24Z

    Switch to branch that has TR tests

commit 07acea8e98ca762d06cc61affad371049a49943b
Author: Alex Petrov <al...@...>
Date:   2018-09-05T15:14:01Z

    Use Sets#difference instead of per-element equality

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by ifesdjeen <gi...@git.apache.org>.
Github user ifesdjeen commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/257#discussion_r215776606
  
    --- Diff: src/java/org/apache/cassandra/repair/LocalSyncTask.java ---
    @@ -39,43 +40,60 @@
     import org.apache.cassandra.tracing.TraceState;
     import org.apache.cassandra.tracing.Tracing;
     import org.apache.cassandra.utils.FBUtilities;
    +import org.apache.cassandra.utils.MerkleTrees;
     
     /**
    - * SymmetricLocalSyncTask performs streaming between local(coordinator) node and remote replica.
    + * LocalSyncTask performs streaming between local(coordinator) node and remote replica.
      */
    -public class SymmetricLocalSyncTask extends SymmetricSyncTask implements StreamEventHandler
    +public class LocalSyncTask extends SymmetricSyncTask implements StreamEventHandler
     {
         private final TraceState state = Tracing.instance.get();
     
    -    private static final Logger logger = LoggerFactory.getLogger(SymmetricLocalSyncTask.class);
    +    private static final Logger logger = LoggerFactory.getLogger(LocalSyncTask.class);
     
    -    private final boolean remoteIsTransient;
         private final UUID pendingRepair;
    -    private final boolean pullRepair;
    +    private final boolean requestRanges;
    +    private final boolean transferRanges;
     
    -    public SymmetricLocalSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, boolean remoteIsTransient, UUID pendingRepair, boolean pullRepair, PreviewKind previewKind)
    +    public LocalSyncTask(RepairJobDesc desc, TreeResponse local, TreeResponse remote, UUID pendingRepair,
    +                         boolean requestRanges, boolean transferRanges, PreviewKind previewKind)
         {
    -        super(desc, r1, r2, previewKind);
    -        this.remoteIsTransient = remoteIsTransient;
    +        this(desc, local.endpoint, remote.endpoint, MerkleTrees.difference(local.trees, remote.trees),
    +             pendingRepair, requestRanges, transferRanges, previewKind);
    +    }
    +
    +    public LocalSyncTask(RepairJobDesc desc, InetAddressAndPort local, InetAddressAndPort remote,
    +                         List<Range<Token>> diff, UUID pendingRepair,
    +                         boolean requestRanges, boolean transferRanges, PreviewKind previewKind)
    +    {
    +        super(desc, local, remote, diff, previewKind);
    +        Preconditions.checkArgument(requestRanges || transferRanges, "Nothing to do in a sync job");
    +        Preconditions.checkArgument(local.equals(FBUtilities.getBroadcastAddressAndPort()));
    +
             this.pendingRepair = pendingRepair;
    -        this.pullRepair = pullRepair;
    +        this.requestRanges = requestRanges;
    +        this.transferRanges = transferRanges;
         }
     
         @VisibleForTesting
    -    StreamPlan createStreamPlan(InetAddressAndPort dst, List<Range<Token>> differences)
    +    StreamPlan createStreamPlan(InetAddressAndPort remote, List<Range<Token>> differences)
         {
             StreamPlan plan = new StreamPlan(StreamOperation.REPAIR, 1, false, pendingRepair, previewKind)
                               .listeners(this)
    -                          .flushBeforeTransfer(pendingRepair == null)
    -                          // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
    -                          .requestRanges(dst, desc.keyspace, RangesAtEndpoint.toDummyList(differences),
    -                                  RangesAtEndpoint.toDummyList(Collections.emptyList()), desc.columnFamily);  // request ranges from the remote node
    +                          .flushBeforeTransfer(pendingRepair == null);
    +
    +        if (requestRanges)
    +        {
    +            // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
    +            plan.requestRanges(remote, desc.keyspace, RangesAtEndpoint.toDummyList(differences),
    +                               RangesAtEndpoint.toDummyList(Collections.emptyList()), desc.columnFamily);
    +        }
     
    -        if (!pullRepair && !remoteIsTransient)
    +        if (transferRanges)
             {
                 // send ranges to the remote node if we are not performing a pull repair
                 // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
    -            plan.transferRanges(dst, desc.keyspace, RangesAtEndpoint.toDummyList(differences), desc.columnFamily);
    +            plan.transferRanges(remote, desc.keyspace, RangesAtEndpoint.toDummyList(differences), desc.columnFamily);
    --- End diff --
    
    I felt like this is a side-effect generating, imperative method, so ignoring its return is fine as it's done in order to facilitate chaining.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by ifesdjeen <gi...@git.apache.org>.
Github user ifesdjeen commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/257#discussion_r215776814
  
    --- Diff: src/java/org/apache/cassandra/service/ActiveRepairService.java ---
    @@ -20,10 +20,25 @@
     import java.io.IOException;
     import java.lang.management.ManagementFactory;
     import java.net.UnknownHostException;
    -import java.util.*;
    -import java.util.concurrent.*;
    +import java.util.ArrayList;
    --- End diff --
    
    Yes, I'll remove it on commit when it's squashed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by ifesdjeen <gi...@git.apache.org>.
Github user ifesdjeen closed the pull request at:

    https://github.com/apache/cassandra/pull/257


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra issue #257: Allow transient range owner to serve as repair coordin...

Posted by bdeggleston <gi...@git.apache.org>.
Github user bdeggleston commented on the issue:

    https://github.com/apache/cassandra/pull/257
  
    @ifesdjeen did some further consolidation of the sync task class hierarchy [here](https://github.com/bdeggleston/cassandra/tree/alexp/tr-followup-1/review2). I'm not really opposed to doing this refactor here instead of a separate ticket, but @krummas should probably take a look now :)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra issue #257: Allow transient range owner to serve as repair coordin...

Posted by bdeggleston <gi...@git.apache.org>.
Github user bdeggleston commented on the issue:

    https://github.com/apache/cassandra/pull/257
  
    I think I've mostly convinced myself this is all you need to do to support coordinating repairs for locally transient ranges. I have a commit [here](https://github.com/bdeggleston/cassandra/tree/alexp/tr-followup-1-updates) here with a few adjustments:
    
    * Move set comparison logic in RepairRunnable#addRangeToNeighbors to CommonRange class 
    * changed asserts to Preconditions
    * added a SymmetricLocalSync task test for the local transient case
    * misc formatting
    
    This leaves the sync task class hierarchy in kind of a messy state. We have AsymmetricSyncTask, but we also have SymmetricSyncTask which can also function as an asymmetric sync task. I do think the state of things is preferable to adding more branches to RepairJob#standardSyncing though, and a refactor is out of scope for this ticket. I'll open a follow on jira to clean that up once this is committed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/257#discussion_r215728753
  
    --- Diff: src/java/org/apache/cassandra/service/ActiveRepairService.java ---
    @@ -20,10 +20,25 @@
     import java.io.IOException;
     import java.lang.management.ManagementFactory;
     import java.net.UnknownHostException;
    -import java.util.*;
    -import java.util.concurrent.*;
    +import java.util.ArrayList;
    --- End diff --
    
    Import fighting still happening?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra issue #257: Allow transient range owner to serve as repair coordin...

Posted by ifesdjeen <gi...@git.apache.org>.
Github user ifesdjeen commented on the issue:

    https://github.com/apache/cassandra/pull/257
  
    @aweisberg fixed imports and spelling. Thank you for spotting those.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by ifesdjeen <gi...@git.apache.org>.
Github user ifesdjeen commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/257#discussion_r215776645
  
    --- Diff: src/java/org/apache/cassandra/repair/RepairJob.java ---
    @@ -165,107 +166,116 @@ private boolean isTransient(InetAddressAndPort ep)
             return session.commonRange.transEndpoints.contains(ep);
         }
     
    -    private AsyncFunction<List<TreeResponse>, List<SyncStat>> standardSyncing()
    +    private ListenableFuture<List<SyncStat>> standardSyncing(List<TreeResponse> trees)
         {
    -        return trees ->
    -        {
    -            InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
    +        InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
     
    -            List<AbstractSyncTask> syncTasks = new ArrayList<>();
    -            // We need to difference all trees one against another
    -            for (int i = 0; i < trees.size() - 1; ++i)
    +        List<AbstractSyncTask> syncTasks = new ArrayList<>();
    +        // We need to difference all trees one against another
    +        for (int i = 0; i < trees.size() - 1; ++i)
    +        {
    +            TreeResponse r1 = trees.get(i);
    +            for (int j = i + 1; j < trees.size(); ++j)
                 {
    -                TreeResponse r1 = trees.get(i);
    -                for (int j = i + 1; j < trees.size(); ++j)
    +                TreeResponse r2 = trees.get(j);
    +
    +                // Avoid streming between two tansient replicas
    +                if (isTransient(r1.endpoint) && isTransient(r2.endpoint))
    +                    continue;
    +
    +                AbstractSyncTask task;
    +                if (r1.endpoint.equals(local) || r2.endpoint.equals(local))
                     {
    -                    TreeResponse r2 = trees.get(j);
    +                    TreeResponse self = r1.endpoint.equals(local) ? r1 : r2;
    +                    TreeResponse remote = r2.endpoint.equals(local) ? r1 : r2;
     
    -                    if (isTransient(r1.endpoint) && isTransient(r2.endpoint))
    +                    // pull only if local is full
    +                    boolean requestRanges = !isTransient(self.endpoint);
    +                    // push only if remote is full; additionally check for pull repair
    +                    boolean transfterRanges = !isTransient(remote.endpoint) && !session.pullRepair;
    --- End diff --
    
    Thanks for noticing! 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by ifesdjeen <gi...@git.apache.org>.
Github user ifesdjeen commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/257#discussion_r215442476
  
    --- Diff: src/java/org/apache/cassandra/repair/RepairRunnable.java ---
    @@ -651,7 +651,7 @@ private static void addRangeToNeighbors(List<CommonRange> neighborRangeList, Ran
             for (CommonRange cr : neighborRangeList)
             {
                 // Use strict equality here, as worst thing that can happen is we generate one more stream
    -            if (Iterables.elementsEqual(cr.endpoints, endpoints) && Iterables.elementsEqual(cr.transEndpoints, transEndpoints))
    +            if (Sets.difference(cr.endpoints, endpoints).isEmpty() && Sets.difference(cr.transEndpoints, transEndpoints).isEmpty())
    --- End diff --
    
    Right, `return s.size() == o.size() && s.containsAll(o);` but with quick hashcode path for non-equality.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra issue #257: Allow transient range owner to serve as repair coordin...

Posted by ifesdjeen <gi...@git.apache.org>.
Github user ifesdjeen commented on the issue:

    https://github.com/apache/cassandra/pull/257
  
    Heh, now I need to take a look as well)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/257#discussion_r215724068
  
    --- Diff: src/java/org/apache/cassandra/repair/LocalSyncTask.java ---
    @@ -39,43 +40,60 @@
     import org.apache.cassandra.tracing.TraceState;
     import org.apache.cassandra.tracing.Tracing;
     import org.apache.cassandra.utils.FBUtilities;
    +import org.apache.cassandra.utils.MerkleTrees;
     
     /**
    - * SymmetricLocalSyncTask performs streaming between local(coordinator) node and remote replica.
    + * LocalSyncTask performs streaming between local(coordinator) node and remote replica.
      */
    -public class SymmetricLocalSyncTask extends SymmetricSyncTask implements StreamEventHandler
    +public class LocalSyncTask extends SymmetricSyncTask implements StreamEventHandler
     {
         private final TraceState state = Tracing.instance.get();
     
    -    private static final Logger logger = LoggerFactory.getLogger(SymmetricLocalSyncTask.class);
    +    private static final Logger logger = LoggerFactory.getLogger(LocalSyncTask.class);
     
    -    private final boolean remoteIsTransient;
         private final UUID pendingRepair;
    -    private final boolean pullRepair;
    +    private final boolean requestRanges;
    +    private final boolean transferRanges;
     
    -    public SymmetricLocalSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, boolean remoteIsTransient, UUID pendingRepair, boolean pullRepair, PreviewKind previewKind)
    +    public LocalSyncTask(RepairJobDesc desc, TreeResponse local, TreeResponse remote, UUID pendingRepair,
    +                         boolean requestRanges, boolean transferRanges, PreviewKind previewKind)
         {
    -        super(desc, r1, r2, previewKind);
    -        this.remoteIsTransient = remoteIsTransient;
    +        this(desc, local.endpoint, remote.endpoint, MerkleTrees.difference(local.trees, remote.trees),
    +             pendingRepair, requestRanges, transferRanges, previewKind);
    +    }
    +
    +    public LocalSyncTask(RepairJobDesc desc, InetAddressAndPort local, InetAddressAndPort remote,
    +                         List<Range<Token>> diff, UUID pendingRepair,
    +                         boolean requestRanges, boolean transferRanges, PreviewKind previewKind)
    +    {
    +        super(desc, local, remote, diff, previewKind);
    +        Preconditions.checkArgument(requestRanges || transferRanges, "Nothing to do in a sync job");
    +        Preconditions.checkArgument(local.equals(FBUtilities.getBroadcastAddressAndPort()));
    +
             this.pendingRepair = pendingRepair;
    -        this.pullRepair = pullRepair;
    +        this.requestRanges = requestRanges;
    +        this.transferRanges = transferRanges;
         }
     
         @VisibleForTesting
    -    StreamPlan createStreamPlan(InetAddressAndPort dst, List<Range<Token>> differences)
    +    StreamPlan createStreamPlan(InetAddressAndPort remote, List<Range<Token>> differences)
         {
             StreamPlan plan = new StreamPlan(StreamOperation.REPAIR, 1, false, pendingRepair, previewKind)
                               .listeners(this)
    -                          .flushBeforeTransfer(pendingRepair == null)
    -                          // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
    -                          .requestRanges(dst, desc.keyspace, RangesAtEndpoint.toDummyList(differences),
    -                                  RangesAtEndpoint.toDummyList(Collections.emptyList()), desc.columnFamily);  // request ranges from the remote node
    +                          .flushBeforeTransfer(pendingRepair == null);
    +
    +        if (requestRanges)
    +        {
    +            // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
    +            plan.requestRanges(remote, desc.keyspace, RangesAtEndpoint.toDummyList(differences),
    +                               RangesAtEndpoint.toDummyList(Collections.emptyList()), desc.columnFamily);
    +        }
     
    -        if (!pullRepair && !remoteIsTransient)
    +        if (transferRanges)
             {
                 // send ranges to the remote node if we are not performing a pull repair
                 // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
    -            plan.transferRanges(dst, desc.keyspace, RangesAtEndpoint.toDummyList(differences), desc.columnFamily);
    +            plan.transferRanges(remote, desc.keyspace, RangesAtEndpoint.toDummyList(differences), desc.columnFamily);
    --- End diff --
    
    So the idiom where it returns this isn't always used to return "this" some of the time it's a copy. Granted here it is returning this, but should you assign the plan field on the stack again?
    
    I don't feel strongly about it just noticing that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/257#discussion_r215434653
  
    --- Diff: src/java/org/apache/cassandra/repair/RepairRunnable.java ---
    @@ -651,7 +651,7 @@ private static void addRangeToNeighbors(List<CommonRange> neighborRangeList, Ran
             for (CommonRange cr : neighborRangeList)
             {
                 // Use strict equality here, as worst thing that can happen is we generate one more stream
    -            if (Iterables.elementsEqual(cr.endpoints, endpoints) && Iterables.elementsEqual(cr.transEndpoints, transEndpoints))
    +            if (Sets.difference(cr.endpoints, endpoints).isEmpty() && Sets.difference(cr.transEndpoints, transEndpoints).isEmpty())
    --- End diff --
    
    You can use .equals() here. Set equality does the right thing.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra issue #257: Allow transient range owner to serve as repair coordin...

Posted by ifesdjeen <gi...@git.apache.org>.
Github user ifesdjeen commented on the issue:

    https://github.com/apache/cassandra/pull/257
  
    @bdeggleston thank you for the review and changes, they look great!
    
    I agree the class hierarchy became not very optimal. I was concentrating on the AssumetrciLocalSyncTask and didn't change the symmetric one. I've added a follow-up commit that makes only one of them necessary and renames it to LocalSyncTask. It did not seem necessary to do the same with remote tasks as there we do not depend on the argument order, which we do in the local case because of transiency. I did try to avoid this change but couldn't find a good way to make it order-independent, but I've changed argument names and made sure checks corresponding are in place.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/257#discussion_r215724675
  
    --- Diff: src/java/org/apache/cassandra/repair/RepairJob.java ---
    @@ -165,107 +166,116 @@ private boolean isTransient(InetAddressAndPort ep)
             return session.commonRange.transEndpoints.contains(ep);
         }
     
    -    private AsyncFunction<List<TreeResponse>, List<SyncStat>> standardSyncing()
    +    private ListenableFuture<List<SyncStat>> standardSyncing(List<TreeResponse> trees)
         {
    -        return trees ->
    -        {
    -            InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
    +        InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
     
    -            List<AbstractSyncTask> syncTasks = new ArrayList<>();
    -            // We need to difference all trees one against another
    -            for (int i = 0; i < trees.size() - 1; ++i)
    +        List<AbstractSyncTask> syncTasks = new ArrayList<>();
    +        // We need to difference all trees one against another
    +        for (int i = 0; i < trees.size() - 1; ++i)
    +        {
    +            TreeResponse r1 = trees.get(i);
    +            for (int j = i + 1; j < trees.size(); ++j)
                 {
    -                TreeResponse r1 = trees.get(i);
    -                for (int j = i + 1; j < trees.size(); ++j)
    +                TreeResponse r2 = trees.get(j);
    +
    +                // Avoid streming between two tansient replicas
    +                if (isTransient(r1.endpoint) && isTransient(r2.endpoint))
    +                    continue;
    +
    +                AbstractSyncTask task;
    +                if (r1.endpoint.equals(local) || r2.endpoint.equals(local))
                     {
    -                    TreeResponse r2 = trees.get(j);
    +                    TreeResponse self = r1.endpoint.equals(local) ? r1 : r2;
    +                    TreeResponse remote = r2.endpoint.equals(local) ? r1 : r2;
     
    -                    if (isTransient(r1.endpoint) && isTransient(r2.endpoint))
    +                    // pull only if local is full
    +                    boolean requestRanges = !isTransient(self.endpoint);
    +                    // push only if remote is full; additionally check for pull repair
    +                    boolean transfterRanges = !isTransient(remote.endpoint) && !session.pullRepair;
    --- End diff --
    
    Typo in transfer


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org