92
92
93
93
import static java .util .Collections .emptyMap ;
94
94
import static java .util .Collections .emptySet ;
95
- import static org .elasticsearch .test . ESTestCase . getPortRange ;
95
+ import static org .elasticsearch .transport . TransportService . NOOP_TRANSPORT_INTERCEPTOR ;
96
96
import static org .hamcrest .Matchers .containsString ;
97
97
import static org .hamcrest .Matchers .empty ;
98
98
import static org .hamcrest .Matchers .equalTo ;
99
99
import static org .hamcrest .Matchers .hasToString ;
100
100
import static org .hamcrest .Matchers .instanceOf ;
101
+ import static org .hamcrest .Matchers .not ;
101
102
import static org .hamcrest .Matchers .notNullValue ;
103
+ import static org .hamcrest .Matchers .nullValue ;
102
104
import static org .hamcrest .Matchers .startsWith ;
103
105
104
106
public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
@@ -186,7 +188,8 @@ public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connecti
186
188
}
187
189
188
190
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 ) {
190
193
Settings updatedSettings = Settings .builder ()
191
194
.put (TransportSettings .PORT .getKey (), getPortRange ())
192
195
.put (settings )
@@ -197,14 +200,19 @@ private MockTransportService buildService(final String name, final Version versi
197
200
}
198
201
Transport transport = build (updatedSettings , version , clusterSettings , doHandshake );
199
202
MockTransportService service = MockTransportService .createNewService (updatedSettings , transport , version , threadPool ,
200
- clusterSettings , Collections .emptySet ());
203
+ clusterSettings , Collections .emptySet (), interceptor );
201
204
service .start ();
202
205
if (acceptRequests ) {
203
206
service .acceptIncomingRequests ();
204
207
}
205
208
return service ;
206
209
}
207
210
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
+
208
216
protected MockTransportService buildService (final String name , final Version version , Settings settings ) {
209
217
return buildService (name , version , null , settings );
210
218
}
@@ -2743,6 +2751,95 @@ public void onConnectionClosed(Transport.Connection connection) {
2743
2751
}
2744
2752
}
2745
2753
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
+
2746
2843
private void closeConnectionChannel (Transport .Connection connection ) {
2747
2844
StubbableTransport .WrappedConnection wrappedConnection = (StubbableTransport .WrappedConnection ) connection ;
2748
2845
TcpTransport .NodeChannels channels = (TcpTransport .NodeChannels ) wrappedConnection .getConnection ();
0 commit comments