Eliaaazzz opened a new pull request, #37355:
URL: https://github.com/apache/beam/pull/37355
This fixes a bug where ByteBuddyDoFnInvokerFactory would return the same
cached invoker for different generic instantiations of the same DoFn class.
Changes:
1. Introduced InvokerCacheKey with TypeDescriptors to ensure unique cache
entries.
2. Updated generateInvokerClass to append type-based hash suffix.
3. Added regression test (testCacheKeyCollisionProof).
Fixes #37351
## Description
This PR addresses a critical bug in the `ByteBuddyDoFnInvokerFactory` where
the caching mechanism for generated `DoFnInvoker` classes failed to distinguish
between different generic instantiations of the same `DoFn` class.
### 1. The Problem: Cache Collision on Generic Types
The previous implementation utilized a `ConcurrentHashMap<Class<?>,
Constructor<?>>` to cache the generated invoker constructors. The cache key was
solely the `DoFn` class itself.
In scenarios involving generic `DoFn` classes (e.g., `class MyGenericFn<T>
extends DoFn<T, T>`), the raw class object is identical regardless of the type
parameter `T`.
* **Scenario:** If a pipeline instantiates `MyGenericFn<String>` and later
`MyGenericFn<Integer>`, the factory would generate an invoker for `String` and
cache it.
* **The Bug:** When `MyGenericFn<Integer>` is requested, the factory hits
the cache using the same `MyGenericFn.class` key and returns the
`String`-specialized invoker.
* **Consequence:** This leads to runtime `ClassCastException` or incorrect
`Coder` inference, as the bytecode for the invoker is specialized for the wrong
type.
### 2. The Solution: Type-Aware Caching & Naming
This PR introduces a composite cache key and updates the class naming
strategy to ensure full isolation between generic types.
#### A. Introduced `InvokerCacheKey`
I replaced the simple `Class<?>` key with a new static inner class
`InvokerCacheKey`. This key encapsulates three components:
1. **`fnClass`**: The raw class of the DoFn.
2. **`inputType`**: The `TypeDescriptor` for the input element.
3. **`outputType`**: The `TypeDescriptor` for the output element.
By implementing `equals()` and `hashCode()` based on all three fields, we
ensure that `MyGenericFn<String>` and `MyGenericFn<Integer>` map to distinct
cache entries.
#### B. Unique Class Naming Strategy
ByteBuddy cannot define two different classes with the exact same name in
the same `ClassLoader`. To support multiple generated invokers for the same raw
`DoFn` class, I modified `generateInvokerClass`.
* **Old Strategy:** `<DoFnClassName>$DoFnInvoker`
* **New Strategy:** `<DoFnClassName>$DoFnInvoker$<TypeHash>`
The suffix now includes a hexadecimal hash derived from the input and output
`TypeDescriptors`. This guarantees that the generated bytecode class names are
unique for each generic instantiation.
## Test Plan
I have added a regression test `testCacheKeyCollisionProof` to
`DoFnInvokersTest.java`.
* **Methodology:** The test defines a local generic class
`DynamicTypeDoFn<T>` and manually forces specific `TypeDescriptor` returns.
* **Verification:** It instantiates the DoFn twice (once for `String`, once
for `Integer`) and asserts that the generated Invoker classes are **not** the
same (`assertNotSame`).
* **Result:** The test passes with the fix (confirming distinct invokers are
generated) and fails without it.
**Existing tests passed:**
* `DoFnInvokersTest`
------------------------
Thank you for your contribution! Follow this checklist to help us
incorporate your contribution quickly and easily:
- [x] Mention the appropriate issue in your description (for example:
`addresses #123`), if applicable. This will automatically add a link to the
pull request in the issue. If you would like the issue to automatically close
on merging the pull request, comment `fixes #<ISSUE NUMBER>` instead.
- [ ] Update `CHANGES.md` with noteworthy changes.
- [ ] If this contribution is large, please file an Apache [Individual
Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
See the [Contributor Guide](https://beam.apache.org/contribute) for more
tips on [how to make review process
smoother](https://github.com/apache/beam/blob/master/CONTRIBUTING.md#make-the-reviewers-job-easier).
To check the build health, please visit
[https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
GitHub Actions Tests Status (on master branch)
------------------------------------------------------------------------------------------------
[](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
[](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
[](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
[](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more
information about GitHub Actions CI or the [workflows
README](https://github.com/apache/beam/blob/master/.github/workflows/README.md)
to see a list of phrases to trigger workflows.
--
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]