cassandra-pr mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ifesdjeen <...@git.apache.org>
Subject [GitHub] cassandra pull request #257: Allow transient range owner to serve as repair ...
Date Thu, 06 Sep 2018 20:59:33 GMT
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


Mime
View raw message