93
93
94
94
import static java .util .Collections .emptyMap ;
95
95
import static java .util .Collections .emptySet ;
96
- import static org .elasticsearch .test . ESTestCase . getPortRange ;
96
+ import static org .elasticsearch .transport . TransportService . NOOP_TRANSPORT_INTERCEPTOR ;
97
97
import static org .hamcrest .Matchers .containsString ;
98
98
import static org .hamcrest .Matchers .empty ;
99
99
import static org .hamcrest .Matchers .equalTo ;
100
100
import static org .hamcrest .Matchers .hasToString ;
101
101
import static org .hamcrest .Matchers .instanceOf ;
102
+ import static org .hamcrest .Matchers .not ;
102
103
import static org .hamcrest .Matchers .notNullValue ;
104
+ import static org .hamcrest .Matchers .nullValue ;
103
105
import static org .hamcrest .Matchers .startsWith ;
104
106
105
107
public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
@@ -187,7 +189,8 @@ public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connecti
187
189
}
188
190
189
191
private MockTransportService buildService (final String name , final Version version , @ Nullable ClusterSettings clusterSettings ,
190
- Settings settings , boolean acceptRequests , boolean doHandshake ) {
192
+ Settings settings , boolean acceptRequests , boolean doHandshake ,
193
+ TransportInterceptor interceptor ) {
191
194
Settings updatedSettings = Settings .builder ()
192
195
.put (TransportSettings .PORT .getKey (), getPortRange ())
193
196
.put (settings )
@@ -198,14 +201,19 @@ private MockTransportService buildService(final String name, final Version versi
198
201
}
199
202
Transport transport = build (updatedSettings , version , clusterSettings , doHandshake );
200
203
MockTransportService service = MockTransportService .createNewService (updatedSettings , transport , version , threadPool ,
201
- clusterSettings , Collections .emptySet ());
204
+ clusterSettings , Collections .emptySet (), interceptor );
202
205
service .start ();
203
206
if (acceptRequests ) {
204
207
service .acceptIncomingRequests ();
205
208
}
206
209
return service ;
207
210
}
208
211
212
+ private MockTransportService buildService (final String name , final Version version , @ Nullable ClusterSettings clusterSettings ,
213
+ Settings settings , boolean acceptRequests , boolean doHandshake ) {
214
+ return buildService (name , version , clusterSettings , settings , acceptRequests , doHandshake , NOOP_TRANSPORT_INTERCEPTOR );
215
+ }
216
+
209
217
protected MockTransportService buildService (final String name , final Version version , Settings settings ) {
210
218
return buildService (name , version , null , settings );
211
219
}
@@ -2747,6 +2755,95 @@ public void onConnectionClosed(Transport.Connection connection) {
2747
2755
}
2748
2756
}
2749
2757
2758
+ // test that the response handler is invoked on a failure to send
2759
+ public void testFailToSend () throws InterruptedException {
2760
+ final RuntimeException failToSendException ;
2761
+ if (randomBoolean ()) {
2762
+ failToSendException = new IllegalStateException ("fail to send" );
2763
+ } else {
2764
+ failToSendException = new TransportException ("fail to send" );
2765
+ }
2766
+ final TransportInterceptor interceptor = new TransportInterceptor () {
2767
+ @ Override
2768
+ public AsyncSender interceptSender (final AsyncSender sender ) {
2769
+ return new AsyncSender () {
2770
+ @ Override
2771
+ public <T extends TransportResponse > void sendRequest (
2772
+ final Transport .Connection connection ,
2773
+ final String action ,
2774
+ final TransportRequest request ,
2775
+ final TransportRequestOptions options ,
2776
+ final TransportResponseHandler <T > handler ) {
2777
+ if ("fail-to-send-action" .equals (action )) {
2778
+ throw failToSendException ;
2779
+ } else {
2780
+ sender .sendRequest (connection , action , request , options , handler );
2781
+ }
2782
+ }
2783
+ };
2784
+ }
2785
+ };
2786
+ try (MockTransportService serviceC = buildService ("TS_C" , CURRENT_VERSION , null , Settings .EMPTY , true , true , interceptor )) {
2787
+ serviceC .start ();
2788
+ serviceC .acceptIncomingRequests ();
2789
+ final CountDownLatch latch = new CountDownLatch (1 );
2790
+ serviceC .connectToNode (
2791
+ serviceA .getLocalDiscoNode (),
2792
+ ConnectionProfile .buildDefaultConnectionProfile (Settings .EMPTY ),
2793
+ new ActionListener <>() {
2794
+ @ Override
2795
+ public void onResponse (final Void v ) {
2796
+ latch .countDown ();
2797
+ }
2798
+
2799
+ @ Override
2800
+ public void onFailure (final Exception e ) {
2801
+ fail (e .getMessage ());
2802
+ }
2803
+ });
2804
+ latch .await ();
2805
+ final AtomicReference <TransportException > te = new AtomicReference <>();
2806
+ final Transport .Connection connection = serviceC .getConnection (nodeA );
2807
+ serviceC .sendRequest (
2808
+ connection ,
2809
+ "fail-to-send-action" ,
2810
+ TransportRequest .Empty .INSTANCE ,
2811
+ TransportRequestOptions .EMPTY ,
2812
+ new TransportResponseHandler <TransportResponse >() {
2813
+ @ Override
2814
+ public void handleResponse (final TransportResponse response ) {
2815
+ fail ("handle response should not be invoked" );
2816
+ }
2817
+
2818
+ @ Override
2819
+ public void handleException (final TransportException exp ) {
2820
+ te .set (exp );
2821
+ }
2822
+
2823
+ @ Override
2824
+ public String executor () {
2825
+ return ThreadPool .Names .SAME ;
2826
+ }
2827
+
2828
+ @ Override
2829
+ public TransportResponse read (final StreamInput in ) {
2830
+ return TransportResponse .Empty .INSTANCE ;
2831
+ }
2832
+ });
2833
+ assertThat (te .get (), not (nullValue ()));
2834
+
2835
+ if (failToSendException instanceof IllegalStateException ) {
2836
+ assertThat (te .get ().getMessage (), equalTo ("failure to send" ));
2837
+ assertThat (te .get ().getCause (), instanceOf (IllegalStateException .class ));
2838
+ assertThat (te .get ().getCause ().getMessage (), equalTo ("fail to send" ));
2839
+ } else {
2840
+ assertThat (te .get ().getMessage (), equalTo ("fail to send" ));
2841
+ assertThat (te .get ().getCause (), nullValue ());
2842
+ }
2843
+ }
2844
+
2845
+ }
2846
+
2750
2847
private void closeConnectionChannel (Transport .Connection connection ) {
2751
2848
StubbableTransport .WrappedConnection wrappedConnection = (StubbableTransport .WrappedConnection ) connection ;
2752
2849
TcpTransport .NodeChannels channels = (TcpTransport .NodeChannels ) wrappedConnection .getConnection ();
0 commit comments