damccorm commented on code in PR #34657: URL: https://github.com/apache/beam/pull/34657#discussion_r2051032346
########## sdks/python/apache_beam/io/gcp/bigquery_file_loads.py: ########## @@ -1101,6 +1101,18 @@ def _load_data( of the load jobs would fail but not other. If any of them fails, then copy jobs are not triggered. """ + self.reshuffle_before_load = not util.is_compat_version_prior_to( + p.options, "2.65.0") + if self.reshuffle_before_load: + # Ensure that TriggerLoadJob retry inputs are deterministic by breaking Review Comment: Mentioned offline The contents of GroupByKey should be deterministic unless there was a trigger that fired twice (in which case everything gets recomputed, there's no determinism at all) The order may not be deterministic; I wonder if that's the issue here. That could potentially be solved for by sorting the files in PartitionFiles (thus making GBK + PartitionFiles collectively deterministic) What are the symptoms you're seeing? > Without reshuffle, if more elements destined for a given destination (key for GroupFilesByTableDestinations) arrived between retries, is there a chance these new files could be materialized for the key, and therefore more files are read by the GroupFilesByTableDestinations.read? The GBK operation shouldn't happen until the first trigger has fired. If more elements arrived late, that indicates that another trigger fired (which would impact the Reshuffle case as well) > https://cloud.google.com/dataflow/docs/concepts/exactly-once#output-delivery mentions best practice for IO's is to add a reshuffle before doing a write with side effects. Functionally, we have a reshuffle as part of our GBK; adding another one will make this more expensive (obviously if we need it for correctness we need it, but we should understand why first) -- 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: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org