2727import java .util .Arrays ;
2828import java .util .Collections ;
2929import java .util .Map ;
30+ import java .util .Optional ;
3031import java .util .stream .Stream ;
32+ import org .apache .beam .sdk .annotations .Internal ;
3133import org .apache .beam .sdk .schemas .annotations .SchemaCaseFormat ;
3234import org .apache .beam .sdk .schemas .annotations .SchemaFieldDescription ;
3335import org .apache .beam .sdk .schemas .annotations .SchemaFieldName ;
4042
4143/** Represents type information for a Java type that will be used to infer a Schema type. */
4244@ AutoValue
43- @ SuppressWarnings ({
44- "nullness" , // TODO(https://github.com/apache/beam/issues/20497)
45- "rawtypes"
46- })
45+ @ Internal
4746public abstract class FieldValueTypeInformation implements Serializable {
4847 /** Optionally returns the field index. */
4948 public abstract @ Nullable Integer getNumber ();
@@ -125,18 +124,23 @@ public static FieldValueTypeInformation forOneOf(
125124 .build ();
126125 }
127126
128- public static FieldValueTypeInformation forField (Field field , int index ) {
129- TypeDescriptor <?> type = TypeDescriptor .of (field .getGenericType ());
127+ public static FieldValueTypeInformation forField (
128+ @ Nullable TypeDescriptor <?> typeDescriptor , Field field , int index ) {
129+ TypeDescriptor <?> type =
130+ Optional .ofNullable (typeDescriptor )
131+ .map (td -> (TypeDescriptor ) td .resolveType (field .getGenericType ()))
132+ // fall back to previous behavior
133+ .orElseGet (() -> TypeDescriptor .of (field .getGenericType ()));
130134 return new AutoValue_FieldValueTypeInformation .Builder ()
131135 .setName (getNameOverride (field .getName (), field ))
132136 .setNumber (getNumberOverride (index , field ))
133137 .setNullable (hasNullableAnnotation (field ))
134138 .setType (type )
135139 .setRawType (type .getRawType ())
136140 .setField (field )
137- .setElementType (getIterableComponentType (field ))
138- .setMapKeyType (getMapKeyType (field ))
139- .setMapValueType (getMapValueType (field ))
141+ .setElementType (getIterableComponentType (type ))
142+ .setMapKeyType (getMapKeyType (type ))
143+ .setMapValueType (getMapValueType (type ))
140144 .setOneOfTypes (Collections .emptyMap ())
141145 .setDescription (getFieldDescription (field ))
142146 .build ();
@@ -185,6 +189,11 @@ public static <T extends AnnotatedElement & Member> String getNameOverride(
185189 }
186190
187191 public static FieldValueTypeInformation forGetter (Method method , int index ) {
192+ return forGetter (null , method , index );
193+ }
194+
195+ public static FieldValueTypeInformation forGetter (
196+ @ Nullable TypeDescriptor <?> typeDescriptor , Method method , int index ) {
188197 String name ;
189198 if (method .getName ().startsWith ("get" )) {
190199 name = ReflectUtils .stripPrefix (method .getName (), "get" );
@@ -194,7 +203,12 @@ public static FieldValueTypeInformation forGetter(Method method, int index) {
194203 throw new RuntimeException ("Getter has wrong prefix " + method .getName ());
195204 }
196205
197- TypeDescriptor <?> type = TypeDescriptor .of (method .getGenericReturnType ());
206+ TypeDescriptor <?> type =
207+ Optional .ofNullable (typeDescriptor )
208+ .map (td -> (TypeDescriptor ) td .resolveType (method .getGenericReturnType ()))
209+ // fall back to previous behavior
210+ .orElseGet (() -> TypeDescriptor .of (method .getGenericReturnType ()));
211+
198212 boolean nullable = hasNullableReturnType (method );
199213 return new AutoValue_FieldValueTypeInformation .Builder ()
200214 .setName (getNameOverride (name , method ))
@@ -253,18 +267,32 @@ private static boolean isNullableAnnotation(Annotation annotation) {
253267 }
254268
255269 public static FieldValueTypeInformation forSetter (Method method ) {
256- return forSetter (method , "set" );
270+ return forSetter (null , method );
257271 }
258272
259273 public static FieldValueTypeInformation forSetter (Method method , String setterPrefix ) {
274+ return forSetter (null , method , setterPrefix );
275+ }
276+
277+ public static FieldValueTypeInformation forSetter (
278+ @ Nullable TypeDescriptor <?> typeDescriptor , Method method ) {
279+ return forSetter (typeDescriptor , method , "set" );
280+ }
281+
282+ public static FieldValueTypeInformation forSetter (
283+ @ Nullable TypeDescriptor <?> typeDescriptor , Method method , String setterPrefix ) {
260284 String name ;
261285 if (method .getName ().startsWith (setterPrefix )) {
262286 name = ReflectUtils .stripPrefix (method .getName (), setterPrefix );
263287 } else {
264288 throw new RuntimeException ("Setter has wrong prefix " + method .getName ());
265289 }
266290
267- TypeDescriptor <?> type = TypeDescriptor .of (method .getGenericParameterTypes ()[0 ]);
291+ TypeDescriptor <?> type =
292+ Optional .ofNullable (typeDescriptor )
293+ .map (td -> (TypeDescriptor ) td .resolveType (method .getGenericParameterTypes ()[0 ]))
294+ // fall back to previous behavior
295+ .orElseGet (() -> TypeDescriptor .of (method .getGenericParameterTypes ()[0 ]));
268296 boolean nullable = hasSingleNullableParameter (method );
269297 return new AutoValue_FieldValueTypeInformation .Builder ()
270298 .setName (name )
@@ -283,10 +311,6 @@ public FieldValueTypeInformation withName(String name) {
283311 return toBuilder ().setName (name ).build ();
284312 }
285313
286- private static FieldValueTypeInformation getIterableComponentType (Field field ) {
287- return getIterableComponentType (TypeDescriptor .of (field .getGenericType ()));
288- }
289-
290314 static @ Nullable FieldValueTypeInformation getIterableComponentType (TypeDescriptor <?> valueType ) {
291315 // TODO: Figure out nullable elements.
292316 TypeDescriptor <?> componentType = ReflectUtils .getIterableComponentType (valueType );
@@ -306,34 +330,23 @@ private static FieldValueTypeInformation getIterableComponentType(Field field) {
306330 .build ();
307331 }
308332
309- // If the Field is a map type, returns the key type, otherwise returns a null reference.
310-
311- private static @ Nullable FieldValueTypeInformation getMapKeyType (Field field ) {
312- return getMapKeyType (TypeDescriptor .of (field .getGenericType ()));
313- }
314-
333+ // If the type is a map type, returns the key type, otherwise returns a null reference.
315334 private static @ Nullable FieldValueTypeInformation getMapKeyType (
316335 TypeDescriptor <?> typeDescriptor ) {
317336 return getMapType (typeDescriptor , 0 );
318337 }
319338
320- // If the Field is a map type, returns the value type, otherwise returns a null reference.
321-
322- private static @ Nullable FieldValueTypeInformation getMapValueType (Field field ) {
323- return getMapType (TypeDescriptor .of (field .getGenericType ()), 1 );
324- }
325-
339+ // If the type is a map type, returns the value type, otherwise returns a null reference.
326340 private static @ Nullable FieldValueTypeInformation getMapValueType (
327341 TypeDescriptor <?> typeDescriptor ) {
328342 return getMapType (typeDescriptor , 1 );
329343 }
330344
331345 // If the Field is a map type, returns the key or value type (0 is key type, 1 is value).
332346 // Otherwise returns a null reference.
333- @ SuppressWarnings ("unchecked" )
334347 private static @ Nullable FieldValueTypeInformation getMapType (
335348 TypeDescriptor <?> valueType , int index ) {
336- TypeDescriptor mapType = ReflectUtils .getMapType (valueType , index );
349+ TypeDescriptor <?> mapType = ReflectUtils .getMapType (valueType , index );
337350 if (mapType == null ) {
338351 return null ;
339352 }
0 commit comments