nbali opened a new issue, #25758:
URL: https://github.com/apache/beam/issues/25758

   ### What happened?
   
   Scenario:
   - Java
   - using Dataflow
   - 2.45.0
   - streaming
   - streaming engine disabled
   - using `GroupIntoBatches.WithShardedKey` (for example 
`BigQueryIO.Write.withAutoSharding`)
   
   The pipeline fails saying it requires Streaming Engine. Based on my 
knowledge it should work without Streaming Engine as well.
   
   1. Using `GroupIntoBatches.WithShardedKey` triggers 
https://github.com/apache/beam/blob/v2.45.0/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java#L554-L558
   2. The referenced `StreamingGroupIntoBatchesWithShardedKeyOverrideFactory` 
replaces the transform with `StreamingGroupIntoBatchesWithShardedKey`
   3. Inside that it calls 
`DataflowRunner.maybeRecordPCollectionWithAutoSharding` 
https://github.com/apache/beam/blob/v2.45.0/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java#L335
   4. ... and finally that runner method contains the code that always fails if 
Streaming Engine isn't enabled. 
https://github.com/apache/beam/blob/v2.45.0/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java#L1697-L1706
   
   I don't think that check is necessary. We already have a 
`DataflowRunner.verifyDoFnSupported` that should cover checking if the used 
state functionality is supported or not. 
   
https://github.com/apache/beam/blob/v2.45.0/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java#L2442-L2487
   
   I can take this issue and contribute a fix if my assumption is right. I 
personally can't really see the reasoning behind that limitation (I also 
checked the original PR https://github.com/apache/beam/pull/13208) - but maybe 
I missed something.
   
   
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [ ] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] Component: IO connector
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Spark Runner
   - [ ] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [X] 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