ian-Liaozy opened a new issue, #36088:
URL: https://github.com/apache/beam/issues/36088

   ### What happened?
   
   **Description:**
   
   When applying `with_exception_handling()` to a `PTransform`, any previously 
applied resource hints (such as fusion-breaking tags) are lost. This prevents 
the use of error handling and resource hints on the same transform, which can 
be critical for pipelines that rely on resource hints for correct execution on 
specialized hardware like TPUs.
   
   **Steps to Reproduce:**
   
   1. Create a pipeline with a `PTransform`.
   2. Apply a resource hint to the `PTransform` using 
`.with_resource_hints(tags='some_tag')`.
   3. Apply `.with_exception_handling()` to the same `PTransform`.
   4. The resource hints on the resulting `PTransform` will be empty.
   
   **Code Example:**
   
   The following test case demonstrates the issue. The assertions fail at HEAD, 
indicating the resource hints are lost:
   
   ```python
     def test_tags_before_with_exception_handling(self):
       with beam.Pipeline(runner.FlumeRunner()) as root:
         ok, unused_errors = (
             root
             | beam.Create([1])
             | beam.Map(lambda x: x)
             .with_resource_hints(tags='test_tag')
             .with_exception_handling()
         )
       pd = ok.producer.transform
       self.assertIsInstance(pd, beam.transforms.core.ParDo)
       # Expected: {'flume:resources:tags:v1': b'test_tag'}
       # Actual @698751825: {}
       self.assertEqual(
           pd.get_resource_hints(), {'flume:resources:tags:v1': b'test_tag'}
       )
   ```
   Reordering .with_exception_handling() and .with_resource_hints() does not 
fix the issue. The hints are still lost:
   ```python
   def test_tags_after_with_exception_handling(self):
       with beam.Pipeline(runner.FlumeRunner()) as root:
         ok, unused_errors = (
             root
             | beam.Create([1])
             | beam.Map(lambda x: x)
             .with_exception_handling()
             .with_resource_hints(tags='test_tag')
         )
       pd = ok.producer.transform
       self.assertIsInstance(pd, beam.transforms.core.ParDo)
       self.assertEqual(
           pd.get_resource_hints(), {'flume:resources:tags:v1': b'test_tag'}
       )
   ```
   
   **Expected Behavior:**
   
   Resource hints applied to a transform should be preserved and propagated to 
the underlying `ParDo` operations, even when `with_exception_handling()` is 
used.
   
   **Actual Behavior:**
   
   Resource hints are dropped when `with_exception_handling()` is used.
   
   
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [x] Component: Python SDK
   - [ ] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] Component: IO connector
   - [ ] Component: Beam YAML
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Infrastructure
   - [ ] Component: Spark Runner
   - [ ] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [ ] Component: Google Cloud Dataflow Runner


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