ahmedabu98 commented on code in PR #32757:
URL: https://github.com/apache/beam/pull/32757#discussion_r1799082439


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java:
##########
@@ -243,4 +255,45 @@ public static TypeDescriptor boxIfPrimitive(TypeDescriptor 
typeDescriptor) {
         ? TypeDescriptor.of(Primitives.wrap(typeDescriptor.getRawType()))
         : typeDescriptor;
   }
+
+  public static <T> Map<Type, Type> getAllBoundTypes(TypeDescriptor<T> 
typeDescriptor) {
+    Map<Type, Type> boundParameters = Maps.newHashMap();
+    TypeDescriptor<?> currentType = typeDescriptor;
+    do {
+      if (currentType.getType() instanceof ParameterizedType) {
+        ParameterizedType parameterizedType = (ParameterizedType) 
currentType.getType();
+        TypeVariable<?>[] typeVariables = 
currentType.getRawType().getTypeParameters();
+        Type[] typeArguments = parameterizedType.getActualTypeArguments();
+        ;
+        if (typeArguments.length != typeVariables.length) {
+          throw new RuntimeException("Unmatching arguments lengths");
+        }
+        for (int i = 0; i < typeVariables.length; ++i) {
+          boundParameters.put(typeVariables[i], typeArguments[i]);
+        }
+      }
+      Type superClass = currentType.getRawType().getGenericSuperclass();
+      if (superClass == null || superClass.equals(Object.class)) {
+        break;
+      }
+      currentType = TypeDescriptor.of(superClass);
+    } while (true);
+    return boundParameters;
+  }
+
+  public static Type resolveType(Type type, Map<Type, Type> boundTypes) {
+    TypeDescriptor<?> typeDescriptor = TypeDescriptor.of(type);
+    if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Iterable.class))
+        || typeDescriptor.isSubtypeOf(TypeDescriptor.of(Map.class))) {
+      // Don't resolve these as we special case map and interable.
+      return type;
+    }
+
+    if (type instanceof TypeVariable) {
+      TypeVariable<?> typeVariable = (TypeVariable<?>) type;
+      return Preconditions.checkArgumentNotNull(boundTypes.get(typeVariable));

Review Comment:
   Does it make sense to fall back on `type` if `boundTypes` doesn't contain it 
for whatever reason? Instead of failing here



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java:
##########
@@ -243,4 +255,45 @@ public static TypeDescriptor boxIfPrimitive(TypeDescriptor 
typeDescriptor) {
         ? TypeDescriptor.of(Primitives.wrap(typeDescriptor.getRawType()))
         : typeDescriptor;
   }
+
+  public static <T> Map<Type, Type> getAllBoundTypes(TypeDescriptor<T> 
typeDescriptor) {
+    Map<Type, Type> boundParameters = Maps.newHashMap();
+    TypeDescriptor<?> currentType = typeDescriptor;
+    do {
+      if (currentType.getType() instanceof ParameterizedType) {
+        ParameterizedType parameterizedType = (ParameterizedType) 
currentType.getType();
+        TypeVariable<?>[] typeVariables = 
currentType.getRawType().getTypeParameters();
+        Type[] typeArguments = parameterizedType.getActualTypeArguments();
+        ;
+        if (typeArguments.length != typeVariables.length) {
+          throw new RuntimeException("Unmatching arguments lengths");

Review Comment:
   nit: include the typeDescriptor in the error message for easier debugging?



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java:
##########
@@ -243,4 +255,45 @@ public static TypeDescriptor boxIfPrimitive(TypeDescriptor 
typeDescriptor) {
         ? TypeDescriptor.of(Primitives.wrap(typeDescriptor.getRawType()))
         : typeDescriptor;
   }
+
+  public static <T> Map<Type, Type> getAllBoundTypes(TypeDescriptor<T> 
typeDescriptor) {

Review Comment:
   nit: add a comment/javadoc for this method?



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java:
##########
@@ -161,7 +162,8 @@ private static boolean matchConstructor(
     // Verify that constructor parameters match (name and type) the inferred 
schema.
     for (Parameter parameter : constructor.getParameters()) {
       FieldValueTypeInformation type = typeMap.get(parameter.getName());
-      if (type == null || type.getRawType() != parameter.getType()) {
+      ;

Review Comment:
   nit: cleanup



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java:
##########
@@ -745,8 +751,8 @@ protected StackManipulation convertList(TypeDescriptor<?> 
type) {
 
     @Override
     protected StackManipulation convertMap(TypeDescriptor<?> type) {
-      final TypeDescriptor keyType = ReflectUtils.getMapType(type, 0);
-      final TypeDescriptor valueType = ReflectUtils.getMapType(type, 1);
+      final TypeDescriptor keyType = ReflectUtils.getMapType(type, 0, 
Collections.emptyMap());

Review Comment:
   nit: might be cleaner to just overload `ReflectUtils.get____Type()` methods



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java:
##########
@@ -85,14 +86,17 @@ enum MethodType {
    * public getter methods, or special annotations on the class.
    */
   public static Schema schemaFromClass(
-      TypeDescriptor<?> typeDescriptor, FieldValueTypeSupplier 
fieldValueTypeSupplier) {
-    return schemaFromClass(typeDescriptor, fieldValueTypeSupplier, new 
HashMap<>());
+      TypeDescriptor<?> typeDescriptor,
+      FieldValueTypeSupplier fieldValueTypeSupplier,
+      Map<Type, Type> boundTypes) {

Review Comment:
   Same with `fieldFromType()` method below



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java:
##########
@@ -88,14 +89,23 @@ public static List<Method> getMethods(Class<?> clazz) {
     return DECLARED_METHODS.computeIfAbsent(
         clazz,
         c -> {
-          return Arrays.stream(c.getDeclaredMethods())
-              .filter(
-                  m -> !m.isBridge()) // Covariant overloads insert bridge 
functions, which we must
-              // ignore.
-              .filter(m -> !Modifier.isPrivate(m.getModifiers()))
-              .filter(m -> !Modifier.isProtected(m.getModifiers()))
-              .filter(m -> !Modifier.isStatic(m.getModifiers()))
-              .collect(Collectors.toList());
+          List<Method> methods = Lists.newArrayList();
+          do {
+            if (c.getPackage() != null && 
c.getPackage().getName().startsWith("java.")) {
+              break; // skip java built-in classes
+            }
+            Arrays.stream(c.getDeclaredMethods())
+                .filter(
+                    m ->
+                        !m.isBridge()) // Covariant overloads insert bridge 
functions, which we must
+                // ignore.
+                .filter(m -> !Modifier.isPrivate(m.getModifiers()))
+                .filter(m -> !Modifier.isProtected(m.getModifiers()))
+                .filter(m -> !Modifier.isStatic(m.getModifiers()))
+                .forEach(methods::add);
+            c = c.getSuperclass();

Review Comment:
   Why this functionality change to get all methods in the superclass chain?



##########
sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java:
##########
@@ -625,4 +626,41 @@ public void testSetterConstructionWithRenamedFields() 
throws NoSuchSchemaExcepti
     assertEquals(
         registry.getFromRowFunction(BeanWithCaseFormat.class).apply(row), 
beanWithCaseFormat);
   }
+
+  @Test
+  public void testRegisterBeamWithTypeParameter() throws NoSuchSchemaException 
{

Review Comment:
   nit: 
   ```suggestion
     public void testRegisterBeanWithTypeParameter() throws 
NoSuchSchemaException {
   ```



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java:
##########
@@ -125,18 +126,20 @@ public static FieldValueTypeInformation forOneOf(
         .build();
   }
 
-  public static FieldValueTypeInformation forField(Field field, int index) {
-    TypeDescriptor<?> type = TypeDescriptor.of(field.getGenericType());
+  public static FieldValueTypeInformation forField(
+      Field field, int index, Map<Type, Type> boundTypes) {

Review Comment:
   Same with some other changes to method signatures in this class: 
`forGetter()` and `forSetter()`



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java:
##########
@@ -125,18 +126,20 @@ public static FieldValueTypeInformation forOneOf(
         .build();
   }
 
-  public static FieldValueTypeInformation forField(Field field, int index) {
-    TypeDescriptor<?> type = TypeDescriptor.of(field.getGenericType());
+  public static FieldValueTypeInformation forField(
+      Field field, int index, Map<Type, Type> boundTypes) {

Review Comment:
   Note that this is a breaking change to a public method (not marked with 
`@Internal`). Are we okay with that?



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java:
##########
@@ -73,8 +73,11 @@
 public class POJOUtils {
 
   public static Schema schemaFromPojoClass(
-      TypeDescriptor<?> typeDescriptor, FieldValueTypeSupplier 
fieldValueTypeSupplier) {
-    return StaticSchemaInference.schemaFromClass(typeDescriptor, 
fieldValueTypeSupplier);
+      TypeDescriptor<?> typeDescriptor,
+      FieldValueTypeSupplier fieldValueTypeSupplier,
+      Map<Type, Type> boundTypes) {

Review Comment:
   Note: breaking change to a public method's signature



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java:
##########
@@ -64,8 +65,11 @@
 public class JavaBeanUtils {
   /** Create a {@link Schema} for a Java Bean class. */
   public static Schema schemaFromJavaBeanClass(
-      TypeDescriptor<?> typeDescriptor, FieldValueTypeSupplier 
fieldValueTypeSupplier) {
-    return StaticSchemaInference.schemaFromClass(typeDescriptor, 
fieldValueTypeSupplier);
+      TypeDescriptor<?> typeDescriptor,
+      FieldValueTypeSupplier fieldValueTypeSupplier,
+      Map<Type, Type> boundTypes) {

Review Comment:
   Another breaking method signature change



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java:
##########
@@ -85,14 +86,17 @@ enum MethodType {
    * public getter methods, or special annotations on the class.
    */
   public static Schema schemaFromClass(
-      TypeDescriptor<?> typeDescriptor, FieldValueTypeSupplier 
fieldValueTypeSupplier) {
-    return schemaFromClass(typeDescriptor, fieldValueTypeSupplier, new 
HashMap<>());
+      TypeDescriptor<?> typeDescriptor,
+      FieldValueTypeSupplier fieldValueTypeSupplier,
+      Map<Type, Type> boundTypes) {

Review Comment:
   Another breaking method signature change



##########
sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java:
##########
@@ -625,4 +626,41 @@ public void testSetterConstructionWithRenamedFields() 
throws NoSuchSchemaExcepti
     assertEquals(
         registry.getFromRowFunction(BeanWithCaseFormat.class).apply(row), 
beanWithCaseFormat);
   }
+
+  @Test
+  public void testRegisterBeamWithTypeParameter() throws NoSuchSchemaException 
{
+    SchemaRegistry registry = SchemaRegistry.createDefault();
+    TypeDescriptor<TestJavaBeans.SimpleParameterizedBean<String, Long, 
Boolean, SimpleBean>>
+        typeDescriptor =
+            new TypeDescriptor<
+                TestJavaBeans.SimpleParameterizedBean<String, Long, Boolean, 
SimpleBean>>() {};
+    Schema schema = registry.getSchema(typeDescriptor);
+
+    final Schema expectedSchema =
+        Schema.builder()
+            .addBooleanField("value1")
+            .addStringField("value2")
+            .addInt64Field("value3")
+            .addRowField("value4", SIMPLE_BEAN_SCHEMA)
+            .build();
+    assertTrue(expectedSchema.equivalent(schema));
+  }
+
+  @Test
+  public void testRegisterBeanWithInheritedTypeParameter() throws 
NoSuchSchemaException {
+    SchemaRegistry registry = SchemaRegistry.createDefault();
+    TypeDescriptor<TestJavaBeans.SimpleParameterizedBeanSubclass<Short>> 
typeDescriptor =
+        new 
TypeDescriptor<TestJavaBeans.SimpleParameterizedBeanSubclass<Short>>() {};
+    Schema schema = registry.getSchema(typeDescriptor);
+
+    final Schema expectedSchema =
+        Schema.builder()
+            .addBooleanField("value1")
+            .addStringField("value2")
+            .addInt64Field("value3")
+            .addRowField("value4", SIMPLE_BEAN_SCHEMA)
+            .addInt16Field("value5")
+            .build();
+    assertTrue(expectedSchema.equivalent(schema));
+  }

Review Comment:
   add tests for collection, map, nested types?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to