-
Notifications
You must be signed in to change notification settings - Fork 13k
/
RpcCompletenessTest.java
441 lines (367 loc) · 15.1 KB
/
RpcCompletenessTest.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
/*
* 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.flink.runtime.rpc;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.runtime.concurrent.Future;
import org.apache.flink.util.ReflectionUtil;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.reflections.Reflections;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.annotation.Annotation;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.lang.reflect.Type;
import java.lang.reflect.TypeVariable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/**
* Test which ensures that all classes of subtype {@link RpcEndpoint} implement
* the methods specified in the generic gateway type argument.
*
* {@code
* RpcEndpoint<GatewayTypeParameter extends RpcGateway>
* }
*
* Note, that the class hierarchy can also be nested. In this case the type argument
* always has to be the first argument, e.g. {@code
*
* // RpcClass needs to implement RpcGatewayClass' methods
* RpcClass extends RpcEndpoint<RpcGatewayClass>
*
* // RpcClass2 or its subclass needs to implement RpcGatewayClass' methods
* RpcClass<GatewayTypeParameter extends RpcGateway,...> extends RpcEndpoint<GatewayTypeParameter>
* RpcClass2 extends RpcClass<RpcGatewayClass,...>
*
* // needless to say, this can even be nested further
* ...
* }
*
*/
public class RpcCompletenessTest extends TestLogger {
private static Logger LOG = LoggerFactory.getLogger(RpcCompletenessTest.class);
private static final Class<?> futureClass = Future.class;
private static final Class<?> timeoutClass = Time.class;
@Test
@SuppressWarnings({"rawtypes", "unchecked"})
public void testRpcCompleteness() {
Reflections reflections = new Reflections("org.apache.flink");
Set<Class<? extends RpcEndpoint>> classes = reflections.getSubTypesOf(RpcEndpoint.class);
Class<? extends RpcEndpoint> c;
mainloop:
for (Class<? extends RpcEndpoint> rpcEndpoint : classes) {
c = rpcEndpoint;
LOG.debug("-------------");
LOG.debug("c: {}", c);
// skip abstract classes
if (Modifier.isAbstract(c.getModifiers())) {
LOG.debug("Skipping abstract class");
continue;
}
// check for type parameter bound to RpcGateway
// skip if one is found because a subclass will provide the concrete argument
TypeVariable<? extends Class<? extends RpcEndpoint>>[] typeParameters = c.getTypeParameters();
LOG.debug("Checking {} parameters.", typeParameters.length);
for (int i = 0; i < typeParameters.length; i++) {
for (Type bound : typeParameters[i].getBounds()) {
LOG.debug("checking bound {} of type parameter {}", bound, typeParameters[i]);
if (bound.toString().equals("interface " + RpcGateway.class.getName())) {
if (i > 0) {
fail("Type parameter for RpcGateway should come first in " + c);
}
LOG.debug("Skipping class with type parameter bound to RpcGateway.");
// Type parameter is bound to RpcGateway which a subclass will provide
continue mainloop;
}
}
}
// check if this class or any super class contains the RpcGateway argument
Class<?> rpcGatewayType;
do {
LOG.debug("checking type argument of class: {}", c);
rpcGatewayType = ReflectionUtil.getTemplateType1(c);
LOG.debug("type argument is: {}", rpcGatewayType);
c = (Class<? extends RpcEndpoint>) c.getSuperclass();
} while (!RpcGateway.class.isAssignableFrom(rpcGatewayType));
LOG.debug("Checking RRC completeness of endpoint '{}' with gateway '{}'",
rpcEndpoint.getSimpleName(), rpcGatewayType.getSimpleName());
checkCompleteness(rpcEndpoint, (Class<? extends RpcGateway>) rpcGatewayType);
}
}
@SuppressWarnings("rawtypes")
private void checkCompleteness(Class<? extends RpcEndpoint> rpcEndpoint, Class<? extends RpcGateway> rpcGateway) {
List<Method> rpcMethodsFromGateway = getRpcMethodsFromGateway(rpcGateway);
Method[] gatewayMethods = rpcMethodsFromGateway.toArray(new Method[rpcMethodsFromGateway.size()]);
Method[] serverMethods = rpcEndpoint.getMethods();
Map<String, Set<Method>> rpcMethods = new HashMap<>();
Set<Method> unmatchedRpcMethods = new HashSet<>();
for (Method serverMethod : serverMethods) {
if (serverMethod.isAnnotationPresent(RpcMethod.class)) {
if (rpcMethods.containsKey(serverMethod.getName())) {
Set<Method> methods = rpcMethods.get(serverMethod.getName());
methods.add(serverMethod);
rpcMethods.put(serverMethod.getName(), methods);
} else {
Set<Method> methods = new HashSet<>();
methods.add(serverMethod);
rpcMethods.put(serverMethod.getName(), methods);
}
unmatchedRpcMethods.add(serverMethod);
}
}
for (Method gatewayMethod : gatewayMethods) {
assertTrue(
"The rpc endpoint " + rpcEndpoint.getName() + " does not contain a RpcMethod " +
"annotated method with the same name and signature " +
generateEndpointMethodSignature(gatewayMethod) + ".",
rpcMethods.containsKey(gatewayMethod.getName()));
checkGatewayMethod(gatewayMethod);
if (!matchGatewayMethodWithEndpoint(gatewayMethod, rpcMethods.get(gatewayMethod.getName()), unmatchedRpcMethods)) {
fail("Could not find a RpcMethod annotated method in rpc endpoint " +
rpcEndpoint.getName() + " matching the rpc gateway method " +
generateEndpointMethodSignature(gatewayMethod) + " defined in the rpc gateway " +
rpcGateway.getName() + ".");
}
}
if (!unmatchedRpcMethods.isEmpty()) {
StringBuilder builder = new StringBuilder();
for (Method unmatchedRpcMethod : unmatchedRpcMethods) {
builder.append(unmatchedRpcMethod).append("\n");
}
fail("The rpc endpoint " + rpcEndpoint.getName() + " contains rpc methods which " +
"are not matched to gateway methods of " + rpcGateway.getName() + ":\n" +
builder.toString());
}
}
/**
* Checks whether the gateway method fulfills the gateway method requirements.
* <ul>
* <li>It checks whether the return type is void or a {@link Future} wrapping the actual result. </li>
* <li>It checks that the method's parameter list contains at most one parameter annotated with {@link RpcTimeout}.</li>
* </ul>
*
* @param gatewayMethod Gateway method to check
*/
private void checkGatewayMethod(Method gatewayMethod) {
if (!gatewayMethod.getReturnType().equals(Void.TYPE)) {
assertTrue(
"The return type of method " + gatewayMethod.getName() + " in the rpc gateway " +
gatewayMethod.getDeclaringClass().getName() + " is non void and not a " +
"future. Non-void return types have to be returned as a future.",
gatewayMethod.getReturnType().equals(futureClass));
}
Annotation[][] parameterAnnotations = gatewayMethod.getParameterAnnotations();
Class<?>[] parameterTypes = gatewayMethod.getParameterTypes();
int rpcTimeoutParameters = 0;
for (int i = 0; i < parameterAnnotations.length; i++) {
if (RpcCompletenessTest.isRpcTimeout(parameterAnnotations[i])) {
assertTrue(
"The rpc timeout has to be of type " + timeoutClass.getName() + ".",
parameterTypes[i].equals(timeoutClass));
rpcTimeoutParameters++;
}
}
assertTrue("The gateway method " + gatewayMethod + " must have at most one RpcTimeout " +
"annotated parameter.", rpcTimeoutParameters <= 1);
}
/**
* Checks whether we find a matching overloaded version for the gateway method among the methods
* with the same name in the rpc endpoint.
*
* @param gatewayMethod Gateway method
* @param endpointMethods Set of rpc methods on the rpc endpoint with the same name as the gateway
* method
* @param unmatchedRpcMethods Set of unmatched rpc methods on the endpoint side (so far)
*/
private boolean matchGatewayMethodWithEndpoint(Method gatewayMethod, Set<Method> endpointMethods, Set<Method> unmatchedRpcMethods) {
for (Method endpointMethod : endpointMethods) {
if (checkMethod(gatewayMethod, endpointMethod)) {
unmatchedRpcMethods.remove(endpointMethod);
return true;
}
}
return false;
}
private boolean checkMethod(Method gatewayMethod, Method endpointMethod) {
Class<?>[] gatewayParameterTypes = gatewayMethod.getParameterTypes();
Annotation[][] gatewayParameterAnnotations = gatewayMethod.getParameterAnnotations();
Class<?>[] endpointParameterTypes = endpointMethod.getParameterTypes();
List<Class<?>> filteredGatewayParameterTypes = new ArrayList<>();
assertEquals(gatewayParameterTypes.length, gatewayParameterAnnotations.length);
// filter out the RpcTimeout parameters
for (int i = 0; i < gatewayParameterTypes.length; i++) {
if (!RpcCompletenessTest.isRpcTimeout(gatewayParameterAnnotations[i])) {
filteredGatewayParameterTypes.add(gatewayParameterTypes[i]);
}
}
if (filteredGatewayParameterTypes.size() != endpointParameterTypes.length) {
return false;
} else {
// check the parameter types
for (int i = 0; i < filteredGatewayParameterTypes.size(); i++) {
if (!checkType(filteredGatewayParameterTypes.get(i), endpointParameterTypes[i])) {
return false;
}
}
// check the return types
if (endpointMethod.getReturnType() == void.class) {
if (gatewayMethod.getReturnType() != void.class) {
return false;
}
} else {
// has return value. The gateway method should be wrapped in a future
Class<?> futureClass = gatewayMethod.getReturnType();
// sanity check that the return type of a gateway method must be void or a future
if (!futureClass.equals(RpcCompletenessTest.futureClass)) {
return false;
} else {
Class<?> valueClass = ReflectionUtil.getTemplateType1(gatewayMethod.getGenericReturnType());
if (endpointMethod.getReturnType().equals(futureClass)) {
Class<?> rpcEndpointValueClass = ReflectionUtil.getTemplateType1(endpointMethod.getGenericReturnType());
// check if we have the same future value types
if (valueClass != null && rpcEndpointValueClass != null && !checkType(valueClass, rpcEndpointValueClass)) {
return false;
}
} else {
if (valueClass != null && !checkType(valueClass, endpointMethod.getReturnType())) {
return false;
}
}
}
}
return gatewayMethod.getName().equals(endpointMethod.getName());
}
}
private boolean checkType(Class<?> firstType, Class<?> secondType) {
Class<?> firstResolvedType;
Class<?> secondResolvedType;
if (firstType.isPrimitive()) {
firstResolvedType = RpcCompletenessTest.resolvePrimitiveType(firstType);
} else {
firstResolvedType = firstType;
}
if (secondType.isPrimitive()) {
secondResolvedType = RpcCompletenessTest.resolvePrimitiveType(secondType);
} else {
secondResolvedType = secondType;
}
return firstResolvedType.equals(secondResolvedType);
}
/**
* Generates from a gateway rpc method signature the corresponding rpc endpoint signature.
*
* For example the {@link RpcTimeout} annotation adds an additional parameter to the gateway
* signature which is not relevant on the server side.
*
* @param method Method to generate the signature string for
* @return String of the respective server side rpc method signature
*/
private String generateEndpointMethodSignature(Method method) {
StringBuilder builder = new StringBuilder();
if (method.getReturnType().equals(Void.TYPE)) {
builder.append("void").append(" ");
} else if (method.getReturnType().equals(futureClass)) {
Class<?> valueClass = ReflectionUtil.getTemplateType1(method.getGenericReturnType());
builder
.append(futureClass.getSimpleName())
.append("<")
.append(valueClass != null ? valueClass.getSimpleName() : "")
.append(">");
if (valueClass != null) {
builder.append("/").append(valueClass.getSimpleName());
}
builder.append(" ");
} else {
return "Invalid rpc method signature.";
}
builder.append(method.getName()).append("(");
Class<?>[] parameterTypes = method.getParameterTypes();
Annotation[][] parameterAnnotations = method.getParameterAnnotations();
assertEquals(parameterTypes.length, parameterAnnotations.length);
for (int i = 0; i < parameterTypes.length; i++) {
// filter out the RpcTimeout parameters
if (!RpcCompletenessTest.isRpcTimeout(parameterAnnotations[i])) {
builder.append(parameterTypes[i].getName());
if (i < parameterTypes.length -1) {
builder.append(", ");
}
}
}
builder.append(")");
return builder.toString();
}
private static boolean isRpcTimeout(Annotation[] annotations) {
for (Annotation annotation : annotations) {
if (annotation.annotationType().equals(RpcTimeout.class)) {
return true;
}
}
return false;
}
/**
* Returns the boxed type for a primitive type.
*
* @param primitveType Primitive type to resolve
* @return Boxed type for the given primitive type
*/
private static Class<?> resolvePrimitiveType(Class<?> primitveType) {
assert primitveType.isPrimitive();
TypeInformation<?> typeInformation = BasicTypeInfo.getInfoFor(primitveType);
if (typeInformation != null) {
return typeInformation.getTypeClass();
} else {
throw new RuntimeException("Could not retrive basic type information for primitive type " + primitveType + '.');
}
}
/**
* Extract all rpc methods defined by the gateway interface
*
* @param interfaceClass the given rpc gateway interface
* @return all methods defined by the given interface
*/
private List<Method> getRpcMethodsFromGateway(Class<? extends RpcGateway> interfaceClass) {
if(!interfaceClass.isInterface()) {
fail(interfaceClass.getName() + " is not a interface");
}
ArrayList<Method> allMethods = new ArrayList<>();
// Methods defined in RpcGateway are native method
if(interfaceClass.equals(RpcGateway.class)) {
return allMethods;
}
// Get all methods declared in current interface
Collections.addAll(allMethods, interfaceClass.getDeclaredMethods());
// Get all method inherited from super interface
for (Class<?> superClass : interfaceClass.getInterfaces()) {
@SuppressWarnings("unchecked")
Class<? extends RpcGateway> gatewayClass = (Class<? extends RpcGateway>) superClass;
allMethods.addAll(getRpcMethodsFromGateway(gatewayClass));
}
return allMethods;
}
}