kennknowles commented on code in PR #37530:
URL: https://github.com/apache/beam/pull/37530#discussion_r3289021107
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java:
##########
@@ -2327,12 +2331,58 @@ 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;
+ }
+
+ // Use TypeDescriptor.resolveType() to extract ValueState's type parameter
+ // This preserves generic type information better than raw Type
manipulation
+ TypeDescriptor<?> valueTypeDescriptor =
+ stateType.resolveType(ValueState.class.getTypeParameters()[0]);
+
+ // Skip if the type has unresolved parameters (e.g., TypeVariable,
WildcardType)
+ if (valueTypeDescriptor.hasUnresolvedParameters()) {
Review Comment:
I'm not sure this is desirable. For example if the type is
`ValueState<Set<T>>` I still think we can suggest using `SetState`.
--
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]