Skip to content

Commit 7ec7786

Browse files
committed
Invoke response handler on failure to send (#53631)
Today it can happen that a transport message fails to send (for example, because a transport interceptor rejects the request). In this case, the response handler is never invoked, which can lead to necessary cleanups not being performed. There are two ways to handle this. One is to expect every callsite that sends a message to try/catch these exceptions and handle them appropriately. The other is merely to invoke the response handler to handle the exception, which is already equipped to handle transport exceptions.
1 parent ab94fc0 commit 7ec7786

File tree

3 files changed

+141
-26
lines changed

3 files changed

+141
-26
lines changed

server/src/main/java/org/elasticsearch/transport/TransportService.java

+34-22
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,6 @@
4949
import org.elasticsearch.core.internal.io.IOUtils;
5050
import org.elasticsearch.node.NodeClosedException;
5151
import org.elasticsearch.tasks.Task;
52-
import org.elasticsearch.tasks.TaskCancelledException;
5352
import org.elasticsearch.tasks.TaskManager;
5453
import org.elasticsearch.threadpool.Scheduler;
5554
import org.elasticsearch.threadpool.ThreadPool;
@@ -573,37 +572,57 @@ public <T extends TransportResponse> TransportFuture<T> submitRequest(DiscoveryN
573572
public <T extends TransportResponse> void sendRequest(final DiscoveryNode node, final String action,
574573
final TransportRequest request,
575574
final TransportResponseHandler<T> handler) {
575+
final Transport.Connection connection;
576576
try {
577-
Transport.Connection connection = getConnection(node);
578-
sendRequest(connection, action, request, TransportRequestOptions.EMPTY, handler);
579-
} catch (NodeNotConnectedException ex) {
577+
connection = getConnection(node);
578+
} catch (final NodeNotConnectedException ex) {
580579
// the caller might not handle this so we invoke the handler
581580
handler.handleException(ex);
581+
return;
582582
}
583+
sendRequest(connection, action, request, TransportRequestOptions.EMPTY, handler);
583584
}
584585

585586
public final <T extends TransportResponse> void sendRequest(final DiscoveryNode node, final String action,
586587
final TransportRequest request,
587588
final TransportRequestOptions options,
588589
TransportResponseHandler<T> handler) {
590+
final Transport.Connection connection;
589591
try {
590-
Transport.Connection connection = getConnection(node);
591-
sendRequest(connection, action, request, options, handler);
592-
} catch (NodeNotConnectedException ex) {
592+
connection = getConnection(node);
593+
} catch (final NodeNotConnectedException ex) {
593594
// the caller might not handle this so we invoke the handler
594595
handler.handleException(ex);
596+
return;
595597
}
598+
sendRequest(connection, action, request, options, handler);
596599
}
597600

601+
/**
602+
* Sends a request on the specified connection. If there is a failure sending the request, the specified handler is invoked.
603+
*
604+
* @param connection the connection to send the request on
605+
* @param action the name of the action
606+
* @param request the request
607+
* @param options the options for this request
608+
* @param handler the response handler
609+
* @param <T> the type of the transport response
610+
*/
598611
public final <T extends TransportResponse> void sendRequest(final Transport.Connection connection, final String action,
599612
final TransportRequest request,
600613
final TransportRequestOptions options,
601614
TransportResponseHandler<T> handler) {
602615
try {
603616
asyncSender.sendRequest(connection, action, request, options, handler);
604-
} catch (NodeNotConnectedException ex) {
617+
} catch (final Exception ex) {
605618
// the caller might not handle this so we invoke the handler
606-
handler.handleException(ex);
619+
final TransportException te;
620+
if (ex instanceof TransportException) {
621+
te = (TransportException) ex;
622+
} else {
623+
te = new TransportException("failure to send", ex);
624+
}
625+
handler.handleException(te);
607626
}
608627
}
609628

@@ -623,13 +642,15 @@ public final <T extends TransportResponse> void sendChildRequest(final Discovery
623642
final TransportRequest request, final Task parentTask,
624643
final TransportRequestOptions options,
625644
final TransportResponseHandler<T> handler) {
645+
final Transport.Connection connection;
626646
try {
627-
Transport.Connection connection = getConnection(node);
628-
sendChildRequest(connection, action, request, parentTask, options, handler);
629-
} catch (NodeNotConnectedException ex) {
647+
connection = getConnection(node);
648+
} catch (final NodeNotConnectedException ex) {
630649
// the caller might not handle this so we invoke the handler
631650
handler.handleException(ex);
651+
return;
632652
}
653+
sendChildRequest(connection, action, request, parentTask, options, handler);
633654
}
634655

635656
public <T extends TransportResponse> void sendChildRequest(final Transport.Connection connection, final String action,
@@ -643,16 +664,7 @@ public <T extends TransportResponse> void sendChildRequest(final Transport.Conne
643664
final TransportRequestOptions options,
644665
final TransportResponseHandler<T> handler) {
645666
request.setParentTask(localNode.getId(), parentTask.getId());
646-
try {
647-
sendRequest(connection, action, request, options, handler);
648-
} catch (TaskCancelledException ex) {
649-
// The parent task is already cancelled - just fail the request
650-
handler.handleException(new TransportException(ex));
651-
} catch (NodeNotConnectedException ex) {
652-
// the caller might not handle this so we invoke the handler
653-
handler.handleException(ex);
654-
}
655-
667+
sendRequest(connection, action, request, options, handler);
656668
}
657669

658670
private <T extends TransportResponse> void sendRequestInternal(final Transport.Connection connection, final String action,

test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java

+7-1
Original file line numberDiff line numberDiff line change
@@ -120,7 +120,13 @@ public static MockNioTransport newMockTransport(Settings settings, Version versi
120120

121121
public static MockTransportService createNewService(Settings settings, Transport transport, Version version, ThreadPool threadPool,
122122
@Nullable ClusterSettings clusterSettings, Set<String> taskHeaders) {
123-
return new MockTransportService(settings, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
123+
return createNewService(settings, transport, version, threadPool, clusterSettings, taskHeaders, NOOP_TRANSPORT_INTERCEPTOR);
124+
}
125+
126+
public static MockTransportService createNewService(Settings settings, Transport transport, Version version, ThreadPool threadPool,
127+
@Nullable ClusterSettings clusterSettings, Set<String> taskHeaders,
128+
TransportInterceptor interceptor) {
129+
return new MockTransportService(settings, transport, threadPool, interceptor,
124130
boundAddress ->
125131
new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), UUIDs.randomBase64UUID(), boundAddress.publishAddress(),
126132
Node.NODE_ATTRIBUTES.getAsMap(settings), DiscoveryNode.getRolesFromSettings(settings), version),

test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java

+100-3
Original file line numberDiff line numberDiff line change
@@ -92,13 +92,15 @@
9292

9393
import static java.util.Collections.emptyMap;
9494
import static java.util.Collections.emptySet;
95-
import static org.elasticsearch.test.ESTestCase.getPortRange;
95+
import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR;
9696
import static org.hamcrest.Matchers.containsString;
9797
import static org.hamcrest.Matchers.empty;
9898
import static org.hamcrest.Matchers.equalTo;
9999
import static org.hamcrest.Matchers.hasToString;
100100
import static org.hamcrest.Matchers.instanceOf;
101+
import static org.hamcrest.Matchers.not;
101102
import static org.hamcrest.Matchers.notNullValue;
103+
import static org.hamcrest.Matchers.nullValue;
102104
import static org.hamcrest.Matchers.startsWith;
103105

104106
public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
@@ -186,7 +188,8 @@ public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connecti
186188
}
187189

188190
private MockTransportService buildService(final String name, final Version version, @Nullable ClusterSettings clusterSettings,
189-
Settings settings, boolean acceptRequests, boolean doHandshake) {
191+
Settings settings, boolean acceptRequests, boolean doHandshake,
192+
TransportInterceptor interceptor) {
190193
Settings updatedSettings = Settings.builder()
191194
.put(TransportSettings.PORT.getKey(), getPortRange())
192195
.put(settings)
@@ -197,14 +200,19 @@ private MockTransportService buildService(final String name, final Version versi
197200
}
198201
Transport transport = build(updatedSettings, version, clusterSettings, doHandshake);
199202
MockTransportService service = MockTransportService.createNewService(updatedSettings, transport, version, threadPool,
200-
clusterSettings, Collections.emptySet());
203+
clusterSettings, Collections.emptySet(), interceptor);
201204
service.start();
202205
if (acceptRequests) {
203206
service.acceptIncomingRequests();
204207
}
205208
return service;
206209
}
207210

211+
private MockTransportService buildService(final String name, final Version version, @Nullable ClusterSettings clusterSettings,
212+
Settings settings, boolean acceptRequests, boolean doHandshake) {
213+
return buildService(name, version, clusterSettings, settings, acceptRequests, doHandshake, NOOP_TRANSPORT_INTERCEPTOR);
214+
}
215+
208216
protected MockTransportService buildService(final String name, final Version version, Settings settings) {
209217
return buildService(name, version, null, settings);
210218
}
@@ -2743,6 +2751,95 @@ public void onConnectionClosed(Transport.Connection connection) {
27432751
}
27442752
}
27452753

2754+
// test that the response handler is invoked on a failure to send
2755+
public void testFailToSend() throws InterruptedException {
2756+
final RuntimeException failToSendException;
2757+
if (randomBoolean()) {
2758+
failToSendException = new IllegalStateException("fail to send");
2759+
} else {
2760+
failToSendException = new TransportException("fail to send");
2761+
}
2762+
final TransportInterceptor interceptor = new TransportInterceptor() {
2763+
@Override
2764+
public AsyncSender interceptSender(final AsyncSender sender) {
2765+
return new AsyncSender() {
2766+
@Override
2767+
public <T extends TransportResponse> void sendRequest(
2768+
final Transport.Connection connection,
2769+
final String action,
2770+
final TransportRequest request,
2771+
final TransportRequestOptions options,
2772+
final TransportResponseHandler<T> handler) {
2773+
if ("fail-to-send-action".equals(action)) {
2774+
throw failToSendException;
2775+
} else {
2776+
sender.sendRequest(connection, action, request, options, handler);
2777+
}
2778+
}
2779+
};
2780+
}
2781+
};
2782+
try (MockTransportService serviceC = buildService("TS_C", CURRENT_VERSION, null, Settings.EMPTY, true, true, interceptor)) {
2783+
serviceC.start();
2784+
serviceC.acceptIncomingRequests();
2785+
final CountDownLatch latch = new CountDownLatch(1);
2786+
serviceC.connectToNode(
2787+
serviceA.getLocalDiscoNode(),
2788+
ConnectionProfile.buildDefaultConnectionProfile(Settings.EMPTY),
2789+
new ActionListener<>() {
2790+
@Override
2791+
public void onResponse(final Void v) {
2792+
latch.countDown();
2793+
}
2794+
2795+
@Override
2796+
public void onFailure(final Exception e) {
2797+
fail(e.getMessage());
2798+
}
2799+
});
2800+
latch.await();
2801+
final AtomicReference<TransportException> te = new AtomicReference<>();
2802+
final Transport.Connection connection = serviceC.getConnection(nodeA);
2803+
serviceC.sendRequest(
2804+
connection,
2805+
"fail-to-send-action",
2806+
TransportRequest.Empty.INSTANCE,
2807+
TransportRequestOptions.EMPTY,
2808+
new TransportResponseHandler<TransportResponse>() {
2809+
@Override
2810+
public void handleResponse(final TransportResponse response) {
2811+
fail("handle response should not be invoked");
2812+
}
2813+
2814+
@Override
2815+
public void handleException(final TransportException exp) {
2816+
te.set(exp);
2817+
}
2818+
2819+
@Override
2820+
public String executor() {
2821+
return ThreadPool.Names.SAME;
2822+
}
2823+
2824+
@Override
2825+
public TransportResponse read(final StreamInput in) {
2826+
return TransportResponse.Empty.INSTANCE;
2827+
}
2828+
});
2829+
assertThat(te.get(), not(nullValue()));
2830+
2831+
if (failToSendException instanceof IllegalStateException) {
2832+
assertThat(te.get().getMessage(), equalTo("failure to send"));
2833+
assertThat(te.get().getCause(), instanceOf(IllegalStateException.class));
2834+
assertThat(te.get().getCause().getMessage(), equalTo("fail to send"));
2835+
} else {
2836+
assertThat(te.get().getMessage(), equalTo("fail to send"));
2837+
assertThat(te.get().getCause(), nullValue());
2838+
}
2839+
}
2840+
2841+
}
2842+
27462843
private void closeConnectionChannel(Transport.Connection connection) {
27472844
StubbableTransport.WrappedConnection wrappedConnection = (StubbableTransport.WrappedConnection) connection;
27482845
TcpTransport.NodeChannels channels = (TcpTransport.NodeChannels) wrappedConnection.getConnection();

0 commit comments

Comments
 (0)