kennknowles commented on code in PR #37530:
URL: https://github.com/apache/beam/pull/37530#discussion_r2775277119


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java:
##########
@@ -2327,12 +2331,77 @@ private static Map<String, 
DoFnSignature.StateDeclaration> analyzeStateDeclarati
           (TypeDescriptor<? extends State>)
               TypeDescriptor.of(fnClazz).resolveType(unresolvedStateType);
 
+      // Warn if ValueState contains a collection type that could benefit from 
specialized state
+      warnIfValueStateContainsCollection(fnClazz, id, stateType);
+
       declarations.put(id, DoFnSignature.StateDeclaration.create(id, field, 
stateType));
     }
 
     return ImmutableMap.copyOf(declarations);
   }
 
+  /**
+   * Warns if a ValueState is declared with a collection type (Map, List, Set) 
that could benefit
+   * from using specialized state types (MapState, BagState, SetState) for 
better performance.
+   */
+  private static void warnIfValueStateContainsCollection(
+      Class<?> fnClazz, String stateId, TypeDescriptor<? extends State> 
stateType) {
+    if (!stateType.isSubtypeOf(TypeDescriptor.of(ValueState.class))) {
+      return;
+    }
+
+    try {
+      // Get the type directly and extract ValueState's type parameter
+      Type type = stateType.getType();
+      if (!(type instanceof ParameterizedType)) {
+        return;
+      }
+
+      // Find ValueState in the type hierarchy and get its type argument
+      Type valueType = null;
+      ParameterizedType pType = (ParameterizedType) type;
+      if (pType.getRawType() == ValueState.class) {
+        valueType = pType.getActualTypeArguments()[0];
+      } else {
+        // For subtypes of ValueState, we need to resolve the type parameter
+        return;
+      }
+
+      if (valueType == null
+          || valueType instanceof java.lang.reflect.TypeVariable
+          || valueType instanceof java.lang.reflect.WildcardType) {
+        // Cannot determine actual type, skip warning
+        return;
+      }
+
+      TypeDescriptor<?> valueTypeDescriptor = TypeDescriptor.of(valueType);
+      Class<?> rawType = valueTypeDescriptor.getRawType();
+
+      String recommendation = null;
+      if (Map.class.isAssignableFrom(rawType)) {
+        recommendation = "MapState";
+      } else if (List.class.isAssignableFrom(rawType)) {
+        recommendation = "BagState or OrderedListState";
+      } else if (java.util.Set.class.isAssignableFrom(rawType)) {
+        recommendation = "SetState";
+      }
+
+      if (recommendation != null) {
+        LOG.warn(
+            "DoFn {} declares ValueState '{}' with type {}. "
+                + "Storing collections in ValueState requires reading and 
writing the entire "
+                + "collection on each access, which can cause performance 
issues. "
+                + "Consider using {} instead for better performance with large 
collections.",
+            fnClazz.getSimpleName(),
+            stateId,
+            rawType.getSimpleName(),
+            recommendation);
+      }
+    } catch (Exception e) {

Review Comment:
   Can we remove this catch-all block? If there is some exception that we truly 
expect, we should catch it, but we shouldn't suppress all exceptions.



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java:
##########
@@ -2327,12 +2331,77 @@ private static Map<String, 
DoFnSignature.StateDeclaration> analyzeStateDeclarati
           (TypeDescriptor<? extends State>)
               TypeDescriptor.of(fnClazz).resolveType(unresolvedStateType);
 
+      // Warn if ValueState contains a collection type that could benefit from 
specialized state
+      warnIfValueStateContainsCollection(fnClazz, id, stateType);
+
       declarations.put(id, DoFnSignature.StateDeclaration.create(id, field, 
stateType));
     }
 
     return ImmutableMap.copyOf(declarations);
   }
 
+  /**
+   * Warns if a ValueState is declared with a collection type (Map, List, Set) 
that could benefit
+   * from using specialized state types (MapState, BagState, SetState) for 
better performance.
+   */
+  private static void warnIfValueStateContainsCollection(
+      Class<?> fnClazz, String stateId, TypeDescriptor<? extends State> 
stateType) {
+    if (!stateType.isSubtypeOf(TypeDescriptor.of(ValueState.class))) {
+      return;
+    }
+
+    try {
+      // Get the type directly and extract ValueState's type parameter
+      Type type = stateType.getType();
+      if (!(type instanceof ParameterizedType)) {
+        return;
+      }
+
+      // Find ValueState in the type hierarchy and get its type argument
+      Type valueType = null;
+      ParameterizedType pType = (ParameterizedType) type;
+      if (pType.getRawType() == ValueState.class) {

Review Comment:
   Can we do this work using methods of TypeDescriptor? When we pull out the 
`Type` we lose information. The TypeDescriptor should have strictly more 
information about actual generic parameters. (that's actually its whole reason 
for existing)



-- 
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