maedhroz commented on code in PR #4428:
URL: https://github.com/apache/cassandra/pull/4428#discussion_r2539726375


##########
src/java/org/apache/cassandra/replication/CoordinatedTransfer.java:
##########
@@ -0,0 +1,586 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.replication;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.streaming.CassandraOutgoingFile;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.RequestFailure;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.NoPayload;
+import org.apache.cassandra.net.RequestCallbackWithFailure;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.StreamException;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamPlan;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.tcm.ClusterMetadata;
+import org.apache.cassandra.tcm.membership.NodeId;
+import org.apache.cassandra.utils.TimeUUID;
+import org.apache.cassandra.utils.concurrent.AsyncFuture;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.FutureCombiner;
+
+/**
+ * A tracked bulk transfer for a single replica set.
+ * <p>
+ * For simplicity, streaming from coordinator to itself instead of copying 
files. This has some perks:
+ * (1) it allows us to import out-of-range SSTables using the same paths, and
+ * (2) it uses the existing lifecycle management to handle crash-safety, so 
don't need to deal with atomic multi-file
+ * copy.
+ * <p>
+ * A transfer happens in a few steps. First, the coordinator streams the 
SSTables to each replica. Replicas store the
+ * streamed transfer in a "pending" location on the filesystem, where it isn't 
visible to reads. Once the coordinator
+ * receives acknowledgements of completed streams from sufficient replicas, 
the coordinator assigns an "activation ID"
+ * for the transfer, and notifies replicas that the pending stream has been 
activated with that ID. Replicas then move
+ * the pending SSTables into the live set, where they're visible to reads, and 
include the "activation ID" in mutation
+ * tracking summaries for reads that would include the new SSTables.
+ */
+public class CoordinatedTransfer
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(CoordinatedTransfer.class);
+
+    String logPrefix()
+    {
+        return String.format("[CoordinatedTransfer #%s]", transferId);
+    }
+
+    final TimeUUID transferId;
+
+    // TODO(expected): Add epoch at time of creation
+    final String keyspace;
+    public final Range<Token> range;
+
+    final ConcurrentMap<InetAddressAndPort, SingleTransferResult> streams;
+
+    final Collection<SSTableReader> sstables;
+
+    final ConsistencyLevel cl;
+
+    final Supplier<MutationId> getActivationId;
+    volatile MutationId activationId = null;
+
+    CoordinatedTransfer(String keyspace, Range<Token> range, Participants 
participants, Collection<SSTableReader> sstables, ConsistencyLevel cl, 
Supplier<MutationId> getActivationId)
+    {
+        this(TimeUUID.Generator.nextTimeUUID(), keyspace, range, participants, 
sstables, cl, getActivationId);
+    }
+
+    @VisibleForTesting
+    CoordinatedTransfer(TimeUUID transferId, Range<Token> range, MutationId 
activationId)
+    {
+        this.transferId = transferId;
+        this.keyspace = null;
+        this.range = range;
+        this.sstables = Collections.emptyList();
+        this.cl = null;
+        this.getActivationId = () -> activationId;
+        this.activationId = activationId;
+        this.streams = new ConcurrentHashMap<>();
+    }
+
+    private CoordinatedTransfer(TimeUUID transferId, String keyspace, 
Range<Token> range, Participants participants, Collection<SSTableReader> 
sstables, ConsistencyLevel cl, Supplier<MutationId> getActivationId)
+    {
+        this.transferId = transferId;
+        this.keyspace = keyspace;
+        this.range = range;
+        this.sstables = sstables;
+        this.cl = cl;
+        this.getActivationId = getActivationId;
+
+        ClusterMetadata cm = ClusterMetadata.current();
+        this.streams = new ConcurrentHashMap<>(participants.size());
+        for (int i = 0; i < participants.size(); i++)
+        {
+            InetAddressAndPort addr = cm.directory.getNodeAddresses(new 
NodeId(participants.get(i))).broadcastAddress;
+            this.streams.put(addr, SingleTransferResult.Init());
+        }
+    }
+
+    void execute()
+    {
+        logger.debug("Executing tracked bulk transfer {}", this);
+        LocalTransfers.instance().save(this);
+        stream();
+    }
+
+    private void stream()
+    {
+        // TODO: Don't stream multiple copies over the WAN, send one copy and 
indicate forwarding
+        List<Future<Void>> streaming = new ArrayList<>(streams.size());
+        for (InetAddressAndPort to : streams.keySet())
+        {
+            Future<Void> stream = LocalTransfers.instance().executor.submit(() 
-> {
+                stream(to);
+                return null;
+            });
+            streaming.add(stream);
+        }
+
+        Future<List<Void>> future = FutureCombiner.allOf(streaming);
+        try
+        {
+            future.get();
+        }
+        catch (InterruptedException | ExecutionException e)
+        {
+            logger.error("{} Failed due to", logPrefix(), e);
+            Throwable cause = e instanceof ExecutionException ? e.getCause() : 
e;
+            throw cause instanceof RuntimeException ? (RuntimeException) cause 
: new RuntimeException(cause);
+        }
+    }
+
+    private boolean sufficient()
+    {
+        AbstractReplicationStrategy ars = 
Keyspace.open(keyspace).getReplicationStrategy();
+        int blockFor = cl.blockFor(ars);
+        int responses = 0;
+        for (Map.Entry<InetAddressAndPort, SingleTransferResult> entry : 
streams.entrySet())
+        {
+            if (entry.getValue().state == 
SingleTransferResult.State.STREAM_COMPLETE)
+                responses++;
+        }
+        return responses >= blockFor;
+    }
+
+    void stream(InetAddressAndPort to)
+    {
+        SingleTransferResult result;
+        try
+        {
+            result = streamTask(to);
+        }
+        catch (StreamException | ExecutionException | InterruptedException | 
TimeoutException e)
+        {
+            Throwable cause = e instanceof ExecutionException ? e.getCause() : 
e;
+            cleanupFailedStream(to, cause);
+            throw cause instanceof RuntimeException ? (RuntimeException) cause 
: new RuntimeException(cause);
+        }
+
+        try
+        {
+            streamComplete(to, result);
+        }
+        catch (ExecutionException | InterruptedException | TimeoutException e)
+        {
+            Throwable cause = e instanceof ExecutionException ? e.getCause() : 
e;
+            throw cause instanceof RuntimeException ? (RuntimeException) cause 
: new RuntimeException(cause);
+        }
+    }
+
+    private Future<?> notifyFailure()
+    {
+        class NotifyFailure extends AsyncFuture<Void> implements 
RequestCallbackWithFailure<NoPayload>
+        {
+            final Set<InetAddressAndPort> responses = 
ConcurrentHashMap.newKeySet(streams.size());
+
+            @Override
+            public void onResponse(Message<NoPayload> msg)
+            {
+                responses.remove(msg.from());
+                if (responses.isEmpty())
+                    trySuccess(null);
+            }
+
+            @Override
+            public void onFailure(InetAddressAndPort from, RequestFailure 
failure)
+            {
+                tryFailure(failure.failure);
+            }
+        };
+
+        NotifyFailure notifyFailure = new NotifyFailure();
+        for (Map.Entry<InetAddressAndPort, SingleTransferResult> entry : 
streams.entrySet())
+        {
+            InetAddressAndPort to = entry.getKey();
+            SingleTransferResult result = entry.getValue();
+            if (result.planId == null)
+                continue;
+
+            logger.debug("Notifying {} of transfer failure for plan {}", to, 
result.planId);
+            notifyFailure.responses.add(to);
+            Message<TransferFailed> msg = 
Message.out(Verb.TRACKED_TRANSFER_FAILED_REQ, new 
TransferFailed(result.planId));
+            MessagingService.instance().sendWithCallback(msg, to, 
notifyFailure);
+        }
+        return notifyFailure;
+    }
+
+    // This shouldn't throw an exception, even if we fail to notify peers of 
the streaming failure
+    private void cleanupFailedStream(InetAddressAndPort to, Throwable cause)
+    {
+        LocalTransfers.instance().scheduleCleanup();
+
+        if (cause instanceof StreamException)
+            streams.get(to).streamFailed(((StreamException) 
cause).finalState.planId);
+        else
+            streams.get(to).streamFailed(null);
+
+        Future<?> notify = notifyFailure();
+        try
+        {
+            notify.get();
+        }
+        catch (Throwable t)
+        {
+            if (cause != null)
+                t.addSuppressed(cause);
+            logger.error("Failed to notify peers of stream failure", t);
+        }
+    }
+
+    private void streamComplete(InetAddressAndPort to, SingleTransferResult 
result) throws ExecutionException, InterruptedException, TimeoutException
+    {
+        streams.put(to, result);
+        logger.info("{} Completed streaming to {}, {}", logPrefix(), to, this);
+        maybeActivate();
+    }
+
+    synchronized void maybeActivate()
+    {
+        // If any activations have already been sent out, send new activations 
to any received plans that have not yet
+        // been activated
+        boolean anyActivated = false;
+        Set<InetAddressAndPort> awaitingActivation = new HashSet<>();
+        for (Map.Entry<InetAddressAndPort, SingleTransferResult> entry : 
streams.entrySet())
+        {
+            InetAddressAndPort peer = entry.getKey();
+            SingleTransferResult result = entry.getValue();
+            if (result.state == SingleTransferResult.State.ACTIVATE_COMPLETE)
+            {
+                anyActivated = true;
+            }
+            else if (result.state == 
SingleTransferResult.State.STREAM_COMPLETE)
+                awaitingActivation.add(peer);
+        }
+        if (anyActivated && !awaitingActivation.isEmpty())
+        {
+            logger.debug("{} Transfer already activated on peers, sending 
activations to {}", logPrefix(), awaitingActivation);
+            activateOn(awaitingActivation);
+            return;
+        }
+
+        // If no activations have been sent out, check whether we have enough 
planIds back to meet the required CL
+        else if (sufficient())
+        {
+            Set<InetAddressAndPort> peers = new HashSet<>();
+            for (Map.Entry<InetAddressAndPort, SingleTransferResult> entry : 
streams.entrySet())
+            {
+                InetAddressAndPort peer = entry.getKey();
+                SingleTransferResult result = entry.getValue();
+                if (result.state == SingleTransferResult.State.STREAM_COMPLETE)
+                    peers.add(peer);
+            }
+            logger.debug("{} Transfer meets consistency level {}, sending 
activations to {}", logPrefix(), cl, peers);
+            activateOn(peers);
+            return;
+        }
+
+        logger.debug("Nothing to activate");
+    }
+
+    void activateOn(Collection<InetAddressAndPort> peers)
+    {
+        Preconditions.checkState(!peers.isEmpty());
+        logger.debug("{} Activating {} on {}", logPrefix(), this, peers);
+
+        if (activationId == null)
+        {
+            activationId = getActivationId.get();
+            logger.info("{} Assigned activationId {}", logPrefix(), 
activationId);
+        }
+        LocalTransfers.instance().activating(this);
+
+        // First phase ensures data is present on disk, then second phase does 
the actual import. This ensures that if
+        // something goes wrong (like a topology change during import), we 
don't have divergence.
+        class Prepare extends AsyncFuture<Void> implements 
RequestCallbackWithFailure<NoPayload>
+        {
+            final Set<InetAddressAndPort> responses = 
ConcurrentHashMap.newKeySet();
+
+            public Prepare()
+            {
+                responses.addAll(peers);
+            }
+
+            @Override
+            public void onResponse(Message<NoPayload> msg)
+            {
+                logger.debug("{} Got response from: {}", logPrefix(), 
msg.from());
+                responses.remove(msg.from());
+                if (responses.isEmpty())
+                    trySuccess(null);
+            }
+
+            @Override
+            public void onFailure(InetAddressAndPort from, RequestFailure 
failure)
+            {
+                logger.debug("{} Got failure {} from {}", logPrefix(), 
failure, from);
+                cleanupFailedStream(from, failure.failure);
+                tryFailure(new RuntimeException("Tracked import failed during 
PREPARE on " + from + " due to " + failure.reason));
+            }
+        }
+
+        Prepare prepare = new Prepare();
+        for (InetAddressAndPort peer : peers)
+        {
+            TransferActivation activation = new TransferActivation(this, peer, 
TransferActivation.Phase.PREPARE);
+            Message<TransferActivation> msg = 
Message.out(Verb.TRACKED_TRANSFER_ACTIVATE_REQ, activation);
+            logger.debug("{} Sending {} to peer {}", logPrefix(), activation, 
peer);
+            MessagingService.instance().sendWithCallback(msg, peer, prepare);
+            SingleTransferResult result = 
CoordinatedTransfer.this.streams.get(msg.from());
+            if (result != null)
+                result.sentActivation();
+        }
+        try
+        {
+            prepare.get();
+        }
+        catch (InterruptedException | ExecutionException e)
+        {
+            Throwable cause = e instanceof ExecutionException ? e.getCause() : 
e;
+            throw cause instanceof RuntimeException ? (RuntimeException) cause 
: new RuntimeException(cause);
+        }
+        logger.debug("{} activation prepare complete for {}", logPrefix(), 
peers);
+
+        // Acknowledgement of activation is equivalent to a remote write 
acknowledgement. The imported SSTables
+        // are now part of the live set, visible to reads.
+        class Commit extends AsyncFuture<Void> implements 
RequestCallbackWithFailure<Void>
+        {
+            final Set<InetAddressAndPort> responses = 
ConcurrentHashMap.newKeySet();
+
+            private Commit(Collection<InetAddressAndPort> peers)
+            {
+                responses.addAll(peers);
+            }
+
+            @Override
+            public void onResponse(Message<Void> msg)
+            {
+                logger.debug("Activation successfully applied on {}", 
msg.from());
+                SingleTransferResult result = 
CoordinatedTransfer.this.streams.get(msg.from());
+                if (result != null)
+                    result.completedActivation();
+
+                
MutationTrackingService.instance.receivedActivationResponse(CoordinatedTransfer.this,
 msg.from());
+                responses.remove(msg.from());
+                if (responses.isEmpty())
+                {
+                    // All activations complete, schedule cleanup to purge 
pending SSTables
+                    LocalTransfers.instance().scheduleCleanup();
+                    trySuccess(null);
+                }
+            }
+
+            @Override
+            public void onFailure(InetAddressAndPort from, RequestFailure 
failure)
+            {
+                logger.error("Failed activation on {} due to {}", from, 
failure);
+                
MutationTrackingService.instance.retryFailedTransfer(CoordinatedTransfer.this, 
from, failure.failure);
+                // TODO(expected): should only fail if we don't meet requested 
CL
+                tryFailure(new RuntimeException("Tracked import failed during 
COMMIT on " + from + " due to " + failure.reason));
+            }
+        }
+
+        Commit commit = new Commit(peers);
+        for (InetAddressAndPort peer : peers)
+        {
+            TransferActivation activation = new TransferActivation(this, peer, 
TransferActivation.Phase.COMMIT);
+            Message<TransferActivation> msg = 
Message.out(Verb.TRACKED_TRANSFER_ACTIVATE_REQ, activation);
+
+            logger.debug("{} Sending {} to peer {}", logPrefix(), activation, 
peer);
+            MessagingService.instance().sendWithCallback(msg, peer, commit);
+        }
+
+        try
+        {
+            commit.get();
+        }
+        catch (InterruptedException | ExecutionException e)
+        {
+            Throwable cause = e instanceof ExecutionException ? e.getCause() : 
e;
+            throw cause instanceof RuntimeException ? (RuntimeException) cause 
: new RuntimeException(cause);
+        }
+        logger.debug("{} activation commit complete for {}", logPrefix(), 
peers);
+    }
+
+    static class SingleTransferResult
+    {
+        enum State
+        {
+            INIT,
+            STREAM_NOOP,
+            STREAM_FAILED,
+            STREAM_COMPLETE,
+            ACTIVATE_START,
+            ACTIVATE_COMPLETE,
+            UNKNOWN;
+        }
+
+        volatile State state;
+        private volatile TimeUUID planId;

Review Comment:
   What do you think about just making the result object immutable? Something 
like...
   
   ```
   static class SingleTransferResult
   {
       enum State
       {
           INIT,
           STREAM_NOOP,
           STREAM_FAILED,
           STREAM_COMPLETE,
           ACTIVATE_START,
           ACTIVATE_COMPLETE;
       }
   
       final State state;
       final TimeUUID planId;
   
       private SingleTransferResult(State state, TimeUUID planId)
       {
           this.state = state;
           this.planId = planId;
       }
   
       // Factory methods for each state transition
       static SingleTransferResult init()
       {
           return new SingleTransferResult(State.INIT, null);
       }
   
       static SingleTransferResult noop()
       {
           return new SingleTransferResult(State.STREAM_NOOP, null);
       }
   
       static SingleTransferResult streamComplete(TimeUUID planId)
       {
           Preconditions.checkNotNull(planId);
           return new SingleTransferResult(State.STREAM_COMPLETE, planId);
       }
   
       static SingleTransferResult streamFailed(TimeUUID planId)
       {
           // planId may be null if failure occurred before streaming started
           return new SingleTransferResult(State.STREAM_FAILED, planId);
       }
   
       SingleTransferResult withActivationSent()
       {
           Preconditions.checkState(state == State.STREAM_COMPLETE);
           return new SingleTransferResult(State.ACTIVATE_START, planId);
       }
   
       SingleTransferResult withActivationComplete()
       {
           Preconditions.checkState(state == State.ACTIVATE_START);
           return new SingleTransferResult(State.ACTIVATE_COMPLETE, planId);
       }
   
       // Preserve planId through state transitions
       SingleTransferResult withFailedActivation()
       {
           return new SingleTransferResult(State.STREAM_FAILED, planId);
       }
   }
   ```
   
   Then usage looks like....
   
   ```
   streams.compute(to, (k, v) -> SingleTransferResult.streamFailed(planId));
   
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to