28
28
import org .apache .hadoop .ipc .protobuf .RpcHeaderProtos .RpcResponseHeaderProto ;
29
29
import org .apache .hadoop .net .NetUtils ;
30
30
import org .apache .hadoop .util .StringUtils ;
31
+ import org .apache .hadoop .util .Time ;
31
32
import org .apache .hadoop .util .concurrent .AsyncGetFuture ;
32
33
import org .junit .Assert ;
33
34
import org .junit .Before ;
38
39
import java .io .IOException ;
39
40
import java .net .InetSocketAddress ;
40
41
import java .util .*;
42
+ import java .util .concurrent .CompletableFuture ;
43
+ import java .util .concurrent .CompletionException ;
41
44
import java .util .concurrent .ExecutionException ;
42
45
import java .util .concurrent .Future ;
43
46
import java .util .concurrent .TimeUnit ;
44
47
import java .util .concurrent .TimeoutException ;
45
48
46
49
import static org .junit .Assert .assertEquals ;
47
50
import static org .junit .Assert .assertFalse ;
51
+ import static org .junit .Assert .assertTrue ;
52
+ import static org .junit .Assert .fail ;
48
53
49
54
public class TestAsyncIPC {
50
55
@@ -137,6 +142,77 @@ void assertReturnValues(long timeout, TimeUnit unit)
137
142
}
138
143
}
139
144
145
+ /**
146
+ * For testing the asynchronous calls of the RPC client
147
+ * implemented with CompletableFuture.
148
+ */
149
+ static class AsyncCompletableFutureCaller extends Thread {
150
+ private final Client client ;
151
+ private final InetSocketAddress server ;
152
+ private final int count ;
153
+ private final List <CompletableFuture <Object >> completableFutures = new ArrayList <>();
154
+ private final List <Long > expectedValues = new ArrayList <>();
155
+
156
+ AsyncCompletableFutureCaller (Client client , InetSocketAddress server , int count ) {
157
+ this .client = client ;
158
+ this .server = server ;
159
+ this .count = count ;
160
+ setName ("Async CompletableFuture Caller" );
161
+ }
162
+
163
+ @ Override
164
+ public void run () {
165
+ // Set the RPC client to use asynchronous mode.
166
+ Client .setAsynchronousMode (true );
167
+ long startTime = Time .monotonicNow ();
168
+ try {
169
+ for (int i = 0 ; i < count ; i ++) {
170
+ final long param = TestIPC .RANDOM .nextLong ();
171
+ // Set the CompletableFuture object for the current Client.Call.
172
+ CompletableFuture <Object > completableFuture = new CompletableFuture <>();
173
+ Client .CALL_FUTURE_THREAD_LOCAL .set (completableFuture );
174
+ // Execute asynchronous call.
175
+ TestIPC .call (client , param , server , conf );
176
+ expectedValues .add (param );
177
+ // After the call is completed, the response thread
178
+ // (currently the Client.connection thread) retrieves the response
179
+ // using the AsyncGetFuture<Writable, IOException> object.
180
+ AsyncGetFuture <Writable , IOException > asyncRpcResponse = getAsyncRpcResponseFuture ();
181
+ completableFuture = completableFuture .thenApply (call -> {
182
+ LOG .info ("[{}] Async response for {}" , Thread .currentThread ().getName (), call );
183
+ assertTrue (Thread .currentThread ().getName ().contains ("connection" ));
184
+ try {
185
+ // Since the current call has already been completed,
186
+ // this method does not need to block.
187
+ return asyncRpcResponse .get ();
188
+ } catch (Exception e ) {
189
+ throw new CompletionException (e );
190
+ }
191
+ });
192
+ completableFutures .add (completableFuture );
193
+ }
194
+ // Since the run method is asynchronous,
195
+ // it does not need to wait for a response after sending a request,
196
+ // so the time taken by the run method is less than count * 100
197
+ // (where 100 is the time taken by the server to process a request).
198
+ long cost = Time .monotonicNow () - startTime ;
199
+ assertTrue (cost < count * 100 );
200
+ LOG .info ("[{}] run cost {}ms" , Thread .currentThread ().getName (), cost );
201
+ } catch (Exception e ) {
202
+ fail ();
203
+ }
204
+ }
205
+
206
+ public void assertReturnValues ()
207
+ throws InterruptedException , ExecutionException {
208
+ for (int i = 0 ; i < count ; i ++) {
209
+ LongWritable value = (LongWritable ) completableFutures .get (i ).get ();
210
+ Assert .assertEquals ("call" + i + " failed." ,
211
+ expectedValues .get (i ).longValue (), value .get ());
212
+ }
213
+ }
214
+ }
215
+
140
216
static class AsyncLimitlCaller extends Thread {
141
217
private Client client ;
142
218
private InetSocketAddress server ;
@@ -538,4 +614,36 @@ public void run() {
538
614
assertEquals (startID + i , callIds .get (i ).intValue ());
539
615
}
540
616
}
617
+
618
+ @ Test (timeout = 60000 )
619
+ public void testAsyncCallWithCompletableFuture () throws IOException ,
620
+ InterruptedException , ExecutionException {
621
+ // Override client to store the call id
622
+ final Client client = new Client (LongWritable .class , conf );
623
+
624
+ // Construct an RPC server, which includes a handler thread.
625
+ final TestServer server = new TestIPC .TestServer (1 , false , conf );
626
+ server .callListener = () -> {
627
+ try {
628
+ // The server requires at least 100 milliseconds to process a request.
629
+ Thread .sleep (100 );
630
+ } catch (InterruptedException e ) {
631
+ throw new RuntimeException (e );
632
+ }
633
+ };
634
+
635
+ try {
636
+ InetSocketAddress addr = NetUtils .getConnectAddress (server );
637
+ server .start ();
638
+ // Send 10 asynchronous requests.
639
+ final AsyncCompletableFutureCaller caller =
640
+ new AsyncCompletableFutureCaller (client , addr , 10 );
641
+ caller .run ();
642
+ // Check if the values returned by the asynchronous call meet the expected values.
643
+ caller .assertReturnValues ();
644
+ } finally {
645
+ client .stop ();
646
+ server .stop ();
647
+ }
648
+ }
541
649
}
0 commit comments