ifesdjeen commented on code in PR #3491:
URL: https://github.com/apache/cassandra/pull/3491#discussion_r1734142055


##########
src/java/org/apache/cassandra/tcm/RemoteProcessor.java:
##########
@@ -201,51 +203,45 @@ public static <REQ, RSP> RSP sendWithCallback(Verb verb, 
REQ request, CandidateI
 
     public static <REQ, RSP> void sendWithCallbackAsync(Promise<RSP> promise, 
Verb verb, REQ request, CandidateIterator candidates, Retry retryPolicy)
     {
-        class Request implements RequestCallbackWithFailure<RSP>
-        {
-            void retry()
-            {
-                if (promise.isCancelled() || promise.isDone())
-                    return;
-                if (Thread.currentThread().isInterrupted())
-                    promise.setFailure(new InterruptedException());
-                if (!candidates.hasNext())
-                    promise.tryFailure(new 
IllegalStateException(String.format("Ran out of candidates while sending %s: 
%s", verb, candidates)));
-
-                MessagingService.instance().sendWithCallback(Message.out(verb, 
request), candidates.next(), this);
-            }
-
-            @Override
-            public void onResponse(Message<RSP> msg)
-            {
-                promise.trySuccess(msg.payload);
-            }
-
-            @Override
-            public void onFailure(InetAddressAndPort from, 
RequestFailureReason reason)
-            {
-                if (reason == RequestFailureReason.NOT_CMS)
-                {
-                    logger.debug("{} is not a member of the CMS, querying it 
to discover current membership", from);
-                    DiscoveredNodes cms = tryDiscover(from);
-                    candidates.addCandidates(cms);
-                    candidates.timeout(from);
-                    logger.debug("Got CMS from {}: {}, retrying on: {}", from, 
cms, candidates);
-                }
-                else
-                {
-                    candidates.timeout(from);
-                    logger.warn("Got error from {}: {} when sending {}, 
retrying on {}", from, reason, verb, candidates);
-                }
-
-                if (retryPolicy.reachedMax())
-                    promise.tryFailure(new 
IllegalStateException(String.format("Could not succeed sending %s to %s after 
%d tries", verb, candidates, retryPolicy.tries)));
-                else
-                    retry();
-            }
-        }
-
-        new Request().retry();
+        //TODO (now): the retry defines how long to wait for a retry, but the 
old behavior scheduled the message right away... should this be delayed as well?
+        MessagingService.instance().<REQ, 
RSP>sendWithRetries(Backoff.fromRetry(retryPolicy), 
MessageDelivery.ImmediateRetryScheduler.instance,
+                                                              verb, request, 
candidates,
+                                                              (attempt, 
success, failure) -> {

Review Comment:
   nit: maybe rename `failure` to throwable?



##########
src/java/org/apache/cassandra/repair/messages/RepairMessage.java:
##########
@@ -155,98 +153,94 @@ public static Supplier<Boolean> always()
 
     public static <T> void sendMessageWithRetries(SharedContext ctx, 
Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, 
InetAddressAndPort endpoint, RequestCallback<T> finalCallback)
     {
-        sendMessageWithRetries(ctx, backoff(ctx, verb), allowRetry, request, 
verb, endpoint, finalCallback, 0);
+        sendMessageWithRetries(ctx, backoff(ctx, verb), allowRetry, request, 
verb, endpoint, finalCallback);
     }
 
     public static <T> void sendMessageWithRetries(SharedContext ctx, 
RepairMessage request, Verb verb, InetAddressAndPort endpoint, 
RequestCallback<T> finalCallback)
     {
-        sendMessageWithRetries(ctx, backoff(ctx, verb), always(), request, 
verb, endpoint, finalCallback, 0);
+        sendMessageWithRetries(ctx, backoff(ctx, verb), always(), request, 
verb, endpoint, finalCallback);
     }
 
     public static void sendMessageWithRetries(SharedContext ctx, RepairMessage 
request, Verb verb, InetAddressAndPort endpoint)
     {
-        sendMessageWithRetries(ctx, backoff(ctx, verb), always(), request, 
verb, endpoint, NOOP_CALLBACK, 0);
+        sendMessageWithRetries(ctx, backoff(ctx, verb), always(), request, 
verb, endpoint, NOOP_CALLBACK);
     }
 
     public static void sendMessageWithRetries(SharedContext ctx, 
Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, 
InetAddressAndPort endpoint)
     {
-        sendMessageWithRetries(ctx, backoff(ctx, verb), allowRetry, request, 
verb, endpoint, NOOP_CALLBACK, 0);
+        sendMessageWithRetries(ctx, backoff(ctx, verb), allowRetry, request, 
verb, endpoint, NOOP_CALLBACK);
     }
 
     @VisibleForTesting
-    static <T> void sendMessageWithRetries(SharedContext ctx, Backoff backoff, 
Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, 
InetAddressAndPort endpoint, RequestCallback<T> finalCallback, int attempt)
+    static <T> void sendMessageWithRetries(SharedContext ctx, Backoff backoff, 
Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, 
InetAddressAndPort endpoint, RequestCallback<T> finalCallback)
     {
         if (!ALLOWS_RETRY.contains(verb))
             throw new AssertionError("Repair verb " + verb + " does not 
support retry, but a request to send with retry was given!");
-        RequestCallback<T> callback = new RequestCallback<>()
-        {
-            @Override
-            public void onResponse(Message<T> msg)
+        BiConsumer<Integer, RequestFailureReason > maybeRecordRetry = 
(attempt, reason) -> {
+            if (attempt <= 0)
+                return;
+            // we don't know what the prefix kind is... so use NONE... this 
impacts logPrefix as it will cause us to use "repair" rather than "preview 
repair" which may not be correct... but close enough...
+            String prefix = 
PreviewKind.NONE.logPrefix(request.parentRepairSession());
+            RepairMetrics.retry(verb, attempt);
+            if (reason == null)
             {
-                maybeRecordRetry(null);
-                finalCallback.onResponse(msg);
+                noSpam.info("{} Retry of repair verb " + verb + " was 
successful after {} attempts", prefix, attempt);
             }
-
-            @Override
-            public void onFailure(InetAddressAndPort from, 
RequestFailureReason failureReason)
+            else if (reason == RequestFailureReason.TIMEOUT)
             {
-                ErrorHandling allowed = errorHandlingSupported(ctx, endpoint, 
verb, request.parentRepairSession());
-                switch (allowed)
-                {
-                    case NONE:
-                        logger.error("[#{}] {} failed on {}: {}", 
request.parentRepairSession(), verb, from, failureReason);
-                        return;
-                    case TIMEOUT:
-                        finalCallback.onFailure(from, failureReason);
-                        return;
-                    case RETRY:
-                        int maxAttempts = backoff.maxAttempts();
-                        if (failureReason == RequestFailureReason.TIMEOUT && 
attempt < maxAttempts && allowRetry.get())
-                        {
-                            ctx.optionalTasks().schedule(() -> 
sendMessageWithRetries(ctx, backoff, allowRetry, request, verb, endpoint, 
finalCallback, attempt + 1),
-                                                         
backoff.computeWaitTime(attempt), backoff.unit());
-                            return;
-                        }
-                        maybeRecordRetry(failureReason);
-                        finalCallback.onFailure(from, failureReason);
-                        return;
-                    default:
-                        throw new AssertionError("Unknown error handler: " + 
allowed);
-                }
+                noSpam.warn("{} Timeout for repair verb " + verb + "; could 
not complete within {} attempts", prefix, attempt);
+                RepairMetrics.retryTimeout(verb);
             }
-
-            private void maybeRecordRetry(@Nullable RequestFailureReason 
reason)
+            else
             {
-                if (attempt <= 0)
-                    return;
-                // we don't know what the prefix kind is... so use NONE... 
this impacts logPrefix as it will cause us to use "repair" rather than "preview 
repair" which may not be correct... but close enough...
-                String prefix = 
PreviewKind.NONE.logPrefix(request.parentRepairSession());
-                RepairMetrics.retry(verb, attempt);
-                if (reason == null)
-                {
-                    noSpam.info("{} Retry of repair verb " + verb + " was 
successful after {} attempts", prefix, attempt);
-                }
-                else if (reason == RequestFailureReason.TIMEOUT)
-                {
-                    noSpam.warn("{} Timeout for repair verb " + verb + "; 
could not complete within {} attempts", prefix, attempt);
-                    RepairMetrics.retryTimeout(verb);
-                }
-                else
-                {
-                    noSpam.warn("{} {} failure for repair verb " + verb + "; 
could not complete within {} attempts", prefix, reason, attempt);
-                    RepairMetrics.retryFailure(verb);
-                }
-            }
-
-            @Override
-            public boolean invokeOnFailure()
-            {
-                return true;
+                noSpam.warn("{} {} failure for repair verb " + verb + "; could 
not complete within {} attempts", prefix, reason, attempt);
+                RepairMetrics.retryFailure(verb);
             }
         };
-        ctx.messaging().sendWithCallback(Message.outWithFlag(verb, request, 
CALL_BACK_ON_FAILURE),
-                                         endpoint,
-                                         callback);
+        ctx.messaging().sendWithRetries(backoff, ctx.optionalTasks()::schedule,
+                                        verb, request, 
Iterators.cycle(endpoint),
+                                        (int attempt, Message<T> msg, 
Throwable failure) -> {
+                                            if (failure == null)
+                                            {
+                                                
maybeRecordRetry.accept(attempt, null);
+                                                finalCallback.onResponse(msg);
+                                            }
+                                        },
+                                        (attempt, from, failure) -> {
+                                            ErrorHandling allowed = 
errorHandlingSupported(ctx, endpoint, verb, request.parentRepairSession());
+                                            switch (allowed)
+                                            {
+                                                case NONE:
+                                                    logger.error("[#{}] {} 
failed on {}: {}", request.parentRepairSession(), verb, from, failure);
+                                                    return false;
+                                                case TIMEOUT:
+                                                    
finalCallback.onFailure(from, failure);
+                                                    return false;
+                                                case RETRY:
+                                                    if (failure == 
RequestFailureReason.TIMEOUT && allowRetry.get())
+                                                        return true;
+                                                    
maybeRecordRetry.accept(attempt, failure);
+                                                    
finalCallback.onFailure(from, failure);
+                                                    return false;
+                                                default:
+                                                    throw new 
AssertionError("Unknown error handler: " + allowed);
+                                            }
+                                        },
+                                        (attempt, retryReason, from, failure) 
-> {
+                                            switch (retryReason)
+                                            {
+                                                case MaxRetries:
+                                                    
maybeRecordRetry.accept(attempt, failure);

Review Comment:
   I might misunderstand, but this lambda seems to be responsible for returning 
an error string rather than controlling a callback. I really like how you kept 
this logic separate in the RemoteProcessor, maybe it's worth to do it here too?



##########
test/unit/org/apache/cassandra/net/SimulatedMessageDelivery.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.net;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.LongSupplier;
+import javax.annotation.Nullable;
+
+import accord.utils.Gens;
+import accord.utils.RandomSource;
+import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.Future;
+
+public class SimulatedMessageDelivery implements MessageDelivery
+{
+    public enum Action { DELIVER, DELIVER_WITH_FAILURE, DROP, 
DROP_PARTITIONED, FAILURE }
+
+    public interface ActionSupplier
+    {
+        Action get(InetAddressAndPort self, Message<?> message, 
InetAddressAndPort to);
+    }
+
+    public interface NetworkDelaySupplier
+    {
+        @Nullable
+        Duration jitter(Message<?> message, InetAddressAndPort to);
+    }
+
+    public static NetworkDelaySupplier noDelay()
+    {
+        return (i1, i2) -> null;
+    }
+
+    public static NetworkDelaySupplier randomDelay(RandomSource rs)
+    {
+        class Connection
+        {
+            final InetAddressAndPort from, to;
+
+            private Connection(InetAddressAndPort from, InetAddressAndPort to)
+            {
+                this.from = from;
+                this.to = to;
+            }
+
+            @Override
+            public boolean equals(Object o)
+            {
+                if (this == o) return true;
+                if (o == null || getClass() != o.getClass()) return false;
+                Connection that = (Connection) o;
+                return from.equals(that.from) && to.equals(that.to);
+            }
+
+            @Override
+            public int hashCode()
+            {
+                return Objects.hash(from, to);
+            }
+
+            @Override
+            public String toString()
+            {
+                return "Connection{" + "from=" + from + ", to=" + to + '}';
+            }
+        }
+        final Map<Connection, LongSupplier> networkLatencies = new HashMap<>();
+        return (msg, to) -> {
+            InetAddressAndPort from = msg.from();
+            long delayNanos = networkLatencies.computeIfAbsent(new 
Connection(from, to), ignore -> {
+                long min = TimeUnit.MICROSECONDS.toNanos(500);
+                long maxSmall = TimeUnit.MILLISECONDS.toNanos(5);
+                long max = TimeUnit.SECONDS.toNanos(5);
+                LongSupplier small = () -> rs.nextLong(min, maxSmall);
+                LongSupplier large = () -> rs.nextLong(maxSmall, max);
+                return Gens.bools().runs(rs.nextInt(1, 11) / 100.0D, 
rs.nextInt(3, 15))
+                           .mapToLong(b -> b ? large.getAsLong() : 
small.getAsLong())
+                           .asLongSupplier(rs.fork());
+            }).getAsLong();
+            return Duration.ofNanos(delayNanos);
+        };
+    }
+
+    public interface Scheduler
+    {
+        void schedule(Runnable command, long delay, TimeUnit unit);
+    }
+
+    public interface DropListener
+    {
+        void onDrop(Action action, InetAddressAndPort from, Message<?> msg);
+    }
+
+    private final InetAddressAndPort self;
+    private final ActionSupplier actions;
+    private final NetworkDelaySupplier networkDelay;
+    private final BiConsumer<InetAddressAndPort, Message<?>> reciever;
+    private final DropListener onDropped;
+    private final Scheduler scheduler;
+    private final Consumer<Throwable> onError;
+    private final Map<CallbackKey, CallbackContext> callbacks = new 
HashMap<>();
+    private enum Status { Up, Down }
+    private Status status = Status.Up;
+
+    public SimulatedMessageDelivery(InetAddressAndPort self,
+                                    ActionSupplier actions,
+                                    NetworkDelaySupplier networkDelay,
+                                    BiConsumer<InetAddressAndPort, Message<?>> 
reciever,
+                                    DropListener onDropped,
+                                    Scheduler scheduler,
+                                    Consumer<Throwable> onError)
+    {
+        this.self = self;
+        this.actions = actions;
+        this.networkDelay = networkDelay;
+        this.reciever = reciever;
+        this.onDropped = onDropped;
+        this.scheduler = scheduler;
+        this.onError = onError;
+    }
+
+    public void stop()
+    {
+        callbacks.clear();
+        status = Status.Down;
+    }
+
+    @Override
+    public <REQ> void send(Message<REQ> message, InetAddressAndPort to)
+    {
+        message = message.withFrom(self);
+        maybeEnqueue(message, to, null);
+    }
+
+    @Override
+    public <REQ, RSP> void sendWithCallback(Message<REQ> message, 
InetAddressAndPort to, RequestCallback<RSP> cb)
+    {
+        message = message.withFrom(self);
+        maybeEnqueue(message, to, cb);
+    }
+
+    @Override
+    public <REQ, RSP> void sendWithCallback(Message<REQ> message, 
InetAddressAndPort to, RequestCallback<RSP> cb, ConnectionType 
specifyConnection)
+    {
+        message = message.withFrom(self);
+        maybeEnqueue(message, to, cb);
+    }
+
+    @Override
+    public <REQ, RSP> Future<Message<RSP>> sendWithResult(Message<REQ> 
message, InetAddressAndPort to)
+    {
+        AsyncPromise<Message<RSP>> promise = new AsyncPromise<>();
+        sendWithCallback(message, to, new RequestCallback<RSP>()
+        {
+            @Override
+            public void onResponse(Message<RSP> msg)
+            {
+                promise.trySuccess(msg);
+            }
+
+            @Override
+            public void onFailure(InetAddressAndPort from, 
RequestFailureReason failure)
+            {
+                promise.tryFailure(new 
MessagingService.FailureResponseException(from, failure));
+            }
+
+            @Override
+            public boolean invokeOnFailure()
+            {
+                return true;
+            }
+        });
+        return promise;
+    }
+
+    @Override
+    public <V> void respond(V response, Message<?> message)
+    {
+        send(message.responseWith(response), message.respondTo());
+    }
+
+    private <REQ, RSP> void maybeEnqueue(Message<REQ> message, 
InetAddressAndPort to, @Nullable RequestCallback<RSP> callback)
+    {
+        if (status != Status.Up)
+            return;
+        CallbackContext cb;
+        if (callback != null)
+        {
+            CallbackKey key = new CallbackKey(message.id(), to);
+            if (callbacks.containsKey(key))
+                throw new AssertionError("Message id " + message.id() + " to " 
+ to + " already has a callback");
+            cb = new CallbackContext(callback);
+            callbacks.put(key, cb);
+        }
+        else
+        {
+            cb = null;
+        }
+        Action action = actions.get(self, message, to);
+        switch (action)
+        {
+            case DELIVER:
+                deliver(message, to);
+                break;
+            case DROP:
+            case DROP_PARTITIONED:
+                onDropped.onDrop(action, to, message);
+                break;
+            case DELIVER_WITH_FAILURE:
+                deliver(message, to);
+            case FAILURE:
+                if (action == Action.FAILURE)
+                    onDropped.onDrop(action, to, message);
+                if (callback != null)
+                    scheduler.schedule(() -> callback.onFailure(to, 
RequestFailureReason.UNKNOWN),
+                                       message.verb().expiresAfterNanos(), 
TimeUnit.NANOSECONDS);
+                return;
+            default:
+                throw new UnsupportedOperationException("Unknown action type: 
" + action);
+        }
+        if (cb != null)
+        {
+            scheduler.schedule(() -> {
+                CallbackContext ctx = callbacks.remove(new 
CallbackKey(message.id(), to));
+                if (ctx != null)
+                {
+                    assert ctx == cb;
+                    try
+                    {
+                        ctx.onFailure(to, RequestFailureReason.TIMEOUT);
+                    }
+                    catch (Throwable t)
+                    {
+                        onError.accept(t);
+                    }
+                }
+            }, message.verb().expiresAfterNanos(), TimeUnit.NANOSECONDS);
+        }
+    }
+
+    private void deliver(Message<?> message, InetAddressAndPort to)
+    {
+        Duration delay = networkDelay.jitter(message, to);
+        if (delay == null) reciever.accept(to, message);
+        else               scheduler.schedule(() -> reciever.accept(to, 
message), delay.toNanos(), TimeUnit.NANOSECONDS);
+    }
+
+    @SuppressWarnings("rawtypes")
+    public SimulatedMessageReceiver reciver(IVerbHandler onMessage)
+    {
+        return new SimulatedMessageReceiver(onMessage);
+    }
+
+    public class SimulatedMessageReceiver
+    {
+        @SuppressWarnings("rawtypes")
+        final IVerbHandler onMessage;
+
+        @SuppressWarnings("rawtypes")
+        public SimulatedMessageReceiver(IVerbHandler onMessage)
+        {
+            this.onMessage = onMessage;
+        }
+
+        public void recieve(Message<?> msg)
+        {
+            if (status != Status.Up)
+                return;
+            if (msg.verb().isResponse())
+            {
+                CallbackKey key = new CallbackKey(msg.id(), msg.from());
+                if (callbacks.containsKey(key))
+                {
+                    CallbackContext callback = callbacks.remove(key);
+                    if (callback == null)
+                        return;
+                    try
+                    {
+                        if (msg.isFailureResponse())
+                            callback.onFailure(msg.from(), 
(RequestFailureReason) msg.payload);
+                        else callback.onResponse(msg);
+                    }
+                    catch (Throwable t)
+                    {
+                        onError.accept(t);
+                    }
+                }
+            }
+            else
+            {
+                try
+                {
+                    //noinspection unchecked
+                    onMessage.doVerb(msg);
+                }
+                catch (Throwable t)
+                {
+                    onError.accept(t);
+                }
+            }
+        }
+    }
+
+    @SuppressWarnings("rawtypes")
+    public static class SimpleVerbHandler implements IVerbHandler

Review Comment:
   nit: unused now. Should we keep this for future usages? 



##########
src/java/org/apache/cassandra/utils/Backoff.java:
##########
@@ -21,23 +21,55 @@
 import java.util.concurrent.TimeUnit;
 import java.util.function.DoubleSupplier;
 
+import org.apache.cassandra.config.RetrySpec;
+import org.apache.cassandra.repair.SharedContext;
+import org.apache.cassandra.tcm.Retry;
+
 public interface Backoff
 {
-    /**
-     * @return max attempts allowed, {@code == 0} implies no retries are 
allowed
-     */
-    int maxAttempts();
-    long computeWaitTime(int retryCount);
+    boolean mayRetry(int attempt);
+    long computeWaitTime(int attempt);
     TimeUnit unit();
 
+    static Backoff fromRetry(Retry retry)

Review Comment:
   Sounds good; if you like - I can take over this one. 



##########
test/unit/org/apache/cassandra/net/SimulatedMessageDelivery.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.net;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.LongSupplier;
+import javax.annotation.Nullable;
+
+import accord.utils.Gens;
+import accord.utils.RandomSource;
+import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.Future;
+
+public class SimulatedMessageDelivery implements MessageDelivery
+{
+    public enum Action { DELIVER, DELIVER_WITH_FAILURE, DROP, 
DROP_PARTITIONED, FAILURE }
+
+    public interface ActionSupplier
+    {
+        Action get(InetAddressAndPort self, Message<?> message, 
InetAddressAndPort to);
+    }
+
+    public interface NetworkDelaySupplier
+    {
+        @Nullable
+        Duration jitter(Message<?> message, InetAddressAndPort to);
+    }
+
+    public static NetworkDelaySupplier noDelay()
+    {
+        return (i1, i2) -> null;
+    }
+
+    public static NetworkDelaySupplier randomDelay(RandomSource rs)
+    {
+        class Connection
+        {
+            final InetAddressAndPort from, to;
+
+            private Connection(InetAddressAndPort from, InetAddressAndPort to)
+            {
+                this.from = from;
+                this.to = to;
+            }
+
+            @Override
+            public boolean equals(Object o)
+            {
+                if (this == o) return true;
+                if (o == null || getClass() != o.getClass()) return false;
+                Connection that = (Connection) o;
+                return from.equals(that.from) && to.equals(that.to);
+            }
+
+            @Override
+            public int hashCode()
+            {
+                return Objects.hash(from, to);
+            }
+
+            @Override
+            public String toString()
+            {
+                return "Connection{" + "from=" + from + ", to=" + to + '}';
+            }
+        }
+        final Map<Connection, LongSupplier> networkLatencies = new HashMap<>();
+        return (msg, to) -> {
+            InetAddressAndPort from = msg.from();
+            long delayNanos = networkLatencies.computeIfAbsent(new 
Connection(from, to), ignore -> {
+                long min = TimeUnit.MICROSECONDS.toNanos(500);
+                long maxSmall = TimeUnit.MILLISECONDS.toNanos(5);
+                long max = TimeUnit.SECONDS.toNanos(5);
+                LongSupplier small = () -> rs.nextLong(min, maxSmall);
+                LongSupplier large = () -> rs.nextLong(maxSmall, max);
+                return Gens.bools().runs(rs.nextInt(1, 11) / 100.0D, 
rs.nextInt(3, 15))
+                           .mapToLong(b -> b ? large.getAsLong() : 
small.getAsLong())
+                           .asLongSupplier(rs.fork());
+            }).getAsLong();
+            return Duration.ofNanos(delayNanos);
+        };
+    }
+
+    public interface Scheduler
+    {
+        void schedule(Runnable command, long delay, TimeUnit unit);
+    }
+
+    public interface DropListener
+    {
+        void onDrop(Action action, InetAddressAndPort from, Message<?> msg);
+    }
+
+    private final InetAddressAndPort self;
+    private final ActionSupplier actions;
+    private final NetworkDelaySupplier networkDelay;
+    private final BiConsumer<InetAddressAndPort, Message<?>> reciever;
+    private final DropListener onDropped;
+    private final Scheduler scheduler;
+    private final Consumer<Throwable> onError;
+    private final Map<CallbackKey, CallbackContext> callbacks = new 
HashMap<>();
+    private enum Status { Up, Down }
+    private Status status = Status.Up;
+
+    public SimulatedMessageDelivery(InetAddressAndPort self,
+                                    ActionSupplier actions,
+                                    NetworkDelaySupplier networkDelay,
+                                    BiConsumer<InetAddressAndPort, Message<?>> 
reciever,
+                                    DropListener onDropped,
+                                    Scheduler scheduler,
+                                    Consumer<Throwable> onError)
+    {
+        this.self = self;
+        this.actions = actions;
+        this.networkDelay = networkDelay;
+        this.reciever = reciever;
+        this.onDropped = onDropped;
+        this.scheduler = scheduler;
+        this.onError = onError;
+    }
+
+    public void stop()
+    {
+        callbacks.clear();
+        status = Status.Down;
+    }
+
+    @Override
+    public <REQ> void send(Message<REQ> message, InetAddressAndPort to)
+    {
+        message = message.withFrom(self);
+        maybeEnqueue(message, to, null);
+    }
+
+    @Override
+    public <REQ, RSP> void sendWithCallback(Message<REQ> message, 
InetAddressAndPort to, RequestCallback<RSP> cb)
+    {
+        message = message.withFrom(self);
+        maybeEnqueue(message, to, cb);
+    }
+
+    @Override
+    public <REQ, RSP> void sendWithCallback(Message<REQ> message, 
InetAddressAndPort to, RequestCallback<RSP> cb, ConnectionType 
specifyConnection)
+    {
+        message = message.withFrom(self);
+        maybeEnqueue(message, to, cb);
+    }
+
+    @Override
+    public <REQ, RSP> Future<Message<RSP>> sendWithResult(Message<REQ> 
message, InetAddressAndPort to)
+    {
+        AsyncPromise<Message<RSP>> promise = new AsyncPromise<>();
+        sendWithCallback(message, to, new RequestCallback<RSP>()
+        {
+            @Override
+            public void onResponse(Message<RSP> msg)
+            {
+                promise.trySuccess(msg);
+            }
+
+            @Override
+            public void onFailure(InetAddressAndPort from, 
RequestFailureReason failure)
+            {
+                promise.tryFailure(new 
MessagingService.FailureResponseException(from, failure));
+            }
+
+            @Override
+            public boolean invokeOnFailure()
+            {
+                return true;
+            }
+        });
+        return promise;
+    }
+
+    @Override
+    public <V> void respond(V response, Message<?> message)
+    {
+        send(message.responseWith(response), message.respondTo());
+    }
+
+    private <REQ, RSP> void maybeEnqueue(Message<REQ> message, 
InetAddressAndPort to, @Nullable RequestCallback<RSP> callback)
+    {
+        if (status != Status.Up)
+            return;
+        CallbackContext cb;
+        if (callback != null)
+        {
+            CallbackKey key = new CallbackKey(message.id(), to);
+            if (callbacks.containsKey(key))
+                throw new AssertionError("Message id " + message.id() + " to " 
+ to + " already has a callback");
+            cb = new CallbackContext(callback);
+            callbacks.put(key, cb);
+        }
+        else
+        {
+            cb = null;
+        }
+        Action action = actions.get(self, message, to);
+        switch (action)
+        {
+            case DELIVER:
+                deliver(message, to);
+                break;
+            case DROP:
+            case DROP_PARTITIONED:
+                onDropped.onDrop(action, to, message);
+                break;
+            case DELIVER_WITH_FAILURE:
+                deliver(message, to);
+            case FAILURE:
+                if (action == Action.FAILURE)
+                    onDropped.onDrop(action, to, message);
+                if (callback != null)
+                    scheduler.schedule(() -> callback.onFailure(to, 
RequestFailureReason.UNKNOWN),
+                                       message.verb().expiresAfterNanos(), 
TimeUnit.NANOSECONDS);
+                return;
+            default:
+                throw new UnsupportedOperationException("Unknown action type: 
" + action);
+        }
+        if (cb != null)
+        {
+            scheduler.schedule(() -> {
+                CallbackContext ctx = callbacks.remove(new 
CallbackKey(message.id(), to));
+                if (ctx != null)
+                {
+                    assert ctx == cb;
+                    try
+                    {
+                        ctx.onFailure(to, RequestFailureReason.TIMEOUT);
+                    }
+                    catch (Throwable t)
+                    {
+                        onError.accept(t);
+                    }
+                }
+            }, message.verb().expiresAfterNanos(), TimeUnit.NANOSECONDS);
+        }
+    }
+
+    private void deliver(Message<?> message, InetAddressAndPort to)
+    {
+        Duration delay = networkDelay.jitter(message, to);
+        if (delay == null) reciever.accept(to, message);
+        else               scheduler.schedule(() -> reciever.accept(to, 
message), delay.toNanos(), TimeUnit.NANOSECONDS);
+    }
+
+    @SuppressWarnings("rawtypes")
+    public SimulatedMessageReceiver reciver(IVerbHandler onMessage)

Review Comment:
   reciver -> receiver



-- 
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