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


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java:
##########
@@ -166,15 +166,67 @@ public <InputT, OutputT> DoFnInvoker<InputT, OutputT> 
invokerFor(DoFn<InputT, Ou
   private static final String FN_DELEGATE_FIELD_NAME = "delegate";
 
   /**
-   * A cache of constructors of generated {@link DoFnInvoker} classes, keyed 
by {@link DoFn} class.
-   * Needed because generating an invoker class is expensive, and to avoid 
generating an excessive
-   * number of classes consuming PermGen memory.
+   * Cache key for DoFnInvoker constructors that includes both the DoFn class 
and its generic type
+   * parameters to prevent collisions when the same DoFn class is used with 
different generic types.
+   */
+  private static final class InvokerCacheKey {
+    private final Class<? extends DoFn<?, ?>> fnClass;
+    private final TypeDescriptor<?> inputType;
+    private final TypeDescriptor<?> outputType;
+
+    InvokerCacheKey(
+        Class<? extends DoFn<?, ?>> fnClass,
+        TypeDescriptor<?> inputType,
+        TypeDescriptor<?> outputType) {
+      this.fnClass = fnClass;
+      this.inputType = inputType;
+      this.outputType = outputType;
+    }
+
+    @Override
+    public boolean equals(@Nullable Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof InvokerCacheKey)) {
+        return false;
+      }
+      InvokerCacheKey that = (InvokerCacheKey) o;
+      return fnClass.equals(that.fnClass)
+          && inputType.equals(that.inputType)
+          && outputType.equals(that.outputType);
+    }
+
+    @Override
+    public int hashCode() {
+      int result = fnClass.hashCode();

Review Comment:
   I suggest using `Objects.hash(fnClass, inputType, outputType)` along with 
`Objects.equals` which makes it very easy to maintain consistency and clarity.



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java:
##########
@@ -166,15 +166,67 @@ public <InputT, OutputT> DoFnInvoker<InputT, OutputT> 
invokerFor(DoFn<InputT, Ou
   private static final String FN_DELEGATE_FIELD_NAME = "delegate";
 
   /**
-   * A cache of constructors of generated {@link DoFnInvoker} classes, keyed 
by {@link DoFn} class.
-   * Needed because generating an invoker class is expensive, and to avoid 
generating an excessive
-   * number of classes consuming PermGen memory.
+   * Cache key for DoFnInvoker constructors that includes both the DoFn class 
and its generic type
+   * parameters to prevent collisions when the same DoFn class is used with 
different generic types.
+   */
+  private static final class InvokerCacheKey {
+    private final Class<? extends DoFn<?, ?>> fnClass;
+    private final TypeDescriptor<?> inputType;
+    private final TypeDescriptor<?> outputType;
+
+    InvokerCacheKey(
+        Class<? extends DoFn<?, ?>> fnClass,
+        TypeDescriptor<?> inputType,
+        TypeDescriptor<?> outputType) {
+      this.fnClass = fnClass;
+      this.inputType = inputType;
+      this.outputType = outputType;
+    }
+
+    @Override
+    public boolean equals(@Nullable Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof InvokerCacheKey)) {
+        return false;
+      }
+      InvokerCacheKey that = (InvokerCacheKey) o;
+      return fnClass.equals(that.fnClass)
+          && inputType.equals(that.inputType)
+          && outputType.equals(that.outputType);
+    }
+
+    @Override
+    public int hashCode() {
+      int result = fnClass.hashCode();
+      result = 31 * result + inputType.hashCode();
+      result = 31 * result + outputType.hashCode();
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return String.format(

Review Comment:
   Here using `ToStringHelper` keeps a consistent format across all our classes.



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java:
##########
@@ -265,11 +317,38 @@ public <InputT, OutputT> DoFnInvoker<InputT, OutputT> 
newByteBuddyInvoker(
         signature.fnClass(),
         fn.getClass());
 
+    // Extract input and output type descriptors from the DoFn instance
+    // Fall back to Object.class if the type descriptors are null or 
unavailable (e.g., MapElements
+    // after serialization)
+    TypeDescriptor<InputT> inputType;
+    try {
+      inputType = fn.getInputTypeDescriptor();
+    } catch (Exception e) {
+      // Some DoFns (like MapElements) throw IllegalStateException if queried 
after

Review Comment:
   I'm interested in digging in to this. Will we still have cache collision 
problems in this case?



##########
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java:
##########
@@ -1382,11 +1385,18 @@ public void process() {}
   @Test
   public void testStableName() {
     DoFnInvoker<Void, Void> invoker = DoFnInvokers.invokerFor(new 
StableNameTestDoFn());
+    // The invoker class name includes a hash of the type descriptors to 
support
+    // different generic instantiations of the same DoFn class.
+    // Format: <DoFn class name>$<DoFnInvoker>$<type hash>
+    TypeDescriptor<Void> voidType = new 
StableNameTestDoFn().getInputTypeDescriptor();
+    String expectedTypeSuffix =

Review Comment:
   Rather than repeat the logic of producing this esoteric suffix, go ahead and 
make it a package-private, or even public method. (the whole module should be 
`@Internal` so public methods are not frozen APIs)



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