dcapwell commented on code in PR #3491:
URL: https://github.com/apache/cassandra/pull/3491#discussion_r1733276864
##########
src/java/org/apache/cassandra/net/MessageDelivery.java:
##########
@@ -74,9 +85,162 @@ public void onFailure(InetAddressAndPort from,
RequestFailureReason reason)
public <REQ, RSP> void sendWithCallback(Message<REQ> message,
InetAddressAndPort to, RequestCallback<RSP> cb);
public <REQ, RSP> void sendWithCallback(Message<REQ> message,
InetAddressAndPort to, RequestCallback<RSP> cb, ConnectionType
specifyConnection);
public <REQ, RSP> Future<Message<RSP>> sendWithResult(Message<REQ>
message, InetAddressAndPort to);
+
+ public default <REQ, RSP> Future<Message<RSP>> sendWithRetries(Backoff
backoff,
+
RetryScheduler retryThreads,
+ Verb verb,
REQ request,
+
InetAddressAndPort candidate,
+
TriFunction<Integer, InetAddressAndPort, RequestFailureReason, Boolean>
shouldRetry,
+
RetryErrorMessage errorMessage)
+ {
+ return sendWithRetries(new AsyncPromise<>(), (Integer i, Message<RSP>
msg) -> msg, backoff, retryThreads, verb, request, Iterators.cycle(candidate),
shouldRetry, errorMessage);
+ }
+
+ public default <REQ, MSG_RSP, RSP> Future<RSP>
sendWithRetries(BiFunction<Integer, Message<MSG_RSP>, RSP> msgToRsp,
+ Backoff
backoff,
+
RetryScheduler retryThreads,
+ Verb verb,
REQ request,
+
InetAddressAndPort candidate,
+
TriFunction<Integer, InetAddressAndPort, RequestFailureReason, Boolean>
shouldRetry,
+
RetryErrorMessage errorMessage)
+ {
+ return sendWithRetries(new AsyncPromise<>(), msgToRsp, backoff,
retryThreads, verb, request, Iterators.cycle(candidate), shouldRetry,
errorMessage);
+ }
+
+ public default <REQ, MSG_RSP, RSP> Future<RSP>
sendWithRetries(Promise<RSP> promise,
+
BiFunction<Integer, Message<MSG_RSP>, RSP> msgToRsp,
+ Backoff
backoff,
+
RetryScheduler retryThreads,
+ Verb verb,
REQ request,
+
Iterator<InetAddressAndPort> candidates,
+
TriFunction<Integer, InetAddressAndPort, RequestFailureReason, Boolean>
shouldRetry,
+
RetryErrorMessage errorMessage)
+ {
+ sendWithRetries(this, promise, msgToRsp, backoff, retryThreads, verb,
request, candidates, shouldRetry, errorMessage, 0);
+ return promise;
+ }
public <V> void respond(V response, Message<?> message);
public default void respondWithFailure(RequestFailureReason reason,
Message<?> message)
{
send(Message.failureResponse(message.id(), message.expiresAtNanos(),
reason), message.respondTo());
}
+
+ interface RetryErrorMessage
+ {
+ String apply(int attempt, ResponseFailureReason retryFailure,
@Nullable InetAddressAndPort from, @Nullable RequestFailureReason reason);
+ }
+
+ private static <REQ, MSG_RSP, RSP> void sendWithRetries(MessageDelivery
messaging,
Review Comment:
pushed
##########
test/unit/org/apache/cassandra/net/SimulatedMessageDelivery.java:
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.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 javax.annotation.Nullable;
+
+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 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 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,
+ BiConsumer<InetAddressAndPort, Message<?>>
reciever,
+ DropListener onDropped,
+ Scheduler scheduler,
+ Consumer<Throwable> onError)
+ {
+ this.self = self;
+ this.actions = actions;
+ 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:
Review Comment:
made the change.. we no longer allocate a promise unless you don't provide a
`OnResult` callback... this is only used by tests atm...
--
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]