-
Notifications
You must be signed in to change notification settings - Fork 4.3k
Add reshuffle before triggering load jobs. #34657
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||||
---|---|---|---|---|---|---|---|---|
|
@@ -44,8 +44,8 @@ | |||||||
from apache_beam.options import value_provider as vp | ||||||||
from apache_beam.options.pipeline_options import GoogleCloudOptions | ||||||||
from apache_beam.transforms import trigger | ||||||||
from apache_beam.transforms import util | ||||||||
from apache_beam.transforms.display import DisplayDataItem | ||||||||
from apache_beam.transforms.util import GroupIntoBatches | ||||||||
from apache_beam.transforms.window import GlobalWindows | ||||||||
|
||||||||
# Protect against environments where bigquery library is not available. | ||||||||
|
@@ -1062,7 +1062,7 @@ def _write_files_with_auto_sharding( | |||||||
destination_data_kv_pc | ||||||||
| | ||||||||
'ToHashableTableRef' >> beam.Map(bigquery_tools.to_hashable_table_ref) | ||||||||
| 'WithAutoSharding' >> GroupIntoBatches.WithShardedKey( | ||||||||
| 'WithAutoSharding' >> util.GroupIntoBatches.WithShardedKey( | ||||||||
batch_size=_FILE_TRIGGERING_RECORD_COUNT, | ||||||||
max_buffering_duration_secs=_FILE_TRIGGERING_BATCHING_DURATION_SECS, | ||||||||
clock=clock) | ||||||||
|
@@ -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 | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Where does the non-determinism currently come from? If I'm reading things correctly, the preceding transform (PartitionFiles) is the only thing between this and the last fusion break (the GBK), and I think that should be deterministic since its operating per-element, but it is possible I'm missing something There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good question, I am not sure exactly where the non-determinism currently come from, but we have seen cases of number of files being uploaded being different between retries during autoscaling and this is the only plausible explanation I could come up with.
Does GroupByKey guarantee determinism for the inputs to PartitionFiles? Without a Reshuffle it looks like part of the GroupFilesByTableDestinations (it lists being part of 3 stages?), PartitionFiles and the TriggerLoadJobs are fused into a single stage. Adding a reshuffle puts TriggerLoadJobs* in their own stages, but it is less obvious what is happening with just the GBK. Java has this precaution Line 822 in 38192de
Line 865 in 38192de
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. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thinking about it more, does Reshuffle force determinism by grouping by unique id's? 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? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 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?
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)
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) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok - after chatting a bit offline my take is that I'm fine with this change if we can have confidence it will fix the issue (the performance cost is vanishingly small since it is just file names being shuffled, not records), but we should have that either empirically or theoretically. @kennknowles may have ideas on what is going on here since it may be related to triggering semantics -
I will be AFK for the next 2 weeks, so don't block on me going forward :) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sg, I did more testing and have been able to consistently reproduce it without this fix, and have not been able to repro it with this fix. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. let us update CHANGES.md to mention this graph change. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Reshuffle checkpoints on Dataflow but not other runners. Otherwise any randomly generated ids in a fused stage will be generated again on retry. Hence this behavior getting the name |
||||||||
# fusion for inputs. | ||||||||
partitions_using_temp_tables = ( | ||||||||
partitions_using_temp_tables | ||||||||
| "ReshuffleBeforeLoadWithTempTables" >> beam.Reshuffle()) | ||||||||
partitions_direct_to_destination = ( | ||||||||
partitions_direct_to_destination | ||||||||
| "ReshuffleBeforeLoadWithoutTempTables" >> beam.Reshuffle()) | ||||||||
|
||||||||
# Load data using temp tables | ||||||||
trigger_loads_outputs = ( | ||||||||
partitions_using_temp_tables | ||||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -478,6 +478,44 @@ def test_records_traverse_transform_with_mocks(self): | |
|
||
assert_that(jobs, equal_to([job_reference]), label='CheckJobs') | ||
|
||
@parameterized.expand([ | ||
param(compat_version=None), | ||
param(compat_version="2.64.0"), | ||
]) | ||
def test_reshuffle_before_load(self, compat_version): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think this really tests what you want to test. You want:
You also want to build an internal dataflow test of update compatibility with requested 2.64.0 version. I can show you a CL that does that if you haven't already seen them. |
||
destination = 'project1:dataset1.table1' | ||
|
||
job_reference = bigquery_api.JobReference() | ||
job_reference.projectId = 'project1' | ||
job_reference.jobId = 'job_name1' | ||
result_job = bigquery_api.Job() | ||
result_job.jobReference = job_reference | ||
|
||
mock_job = mock.Mock() | ||
mock_job.status.state = 'DONE' | ||
mock_job.status.errorResult = None | ||
mock_job.jobReference = job_reference | ||
|
||
bq_client = mock.Mock() | ||
bq_client.jobs.Get.return_value = mock_job | ||
|
||
bq_client.jobs.Insert.return_value = result_job | ||
|
||
transform = bqfl.BigQueryBatchFileLoads( | ||
destination, | ||
custom_gcs_temp_location=self._new_tempdir(), | ||
test_client=bq_client, | ||
validate=False, | ||
temp_file_format=bigquery_tools.FileFormat.JSON) | ||
|
||
options = PipelineOptions(update_compatibility_version=compat_version) | ||
# Need to test this with the DirectRunner to avoid serializing mocks | ||
with TestPipeline('DirectRunner', options=options) as p: | ||
_ = p | beam.Create(_ELEMENTS) | transform | ||
|
||
reshuffle_before_load = compat_version is None | ||
assert transform.reshuffle_before_load == reshuffle_before_load | ||
|
||
def test_load_job_id_used(self): | ||
job_reference = bigquery_api.JobReference() | ||
job_reference.projectId = 'loadJobProject' | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Structural comments:
expand
whenever we move to a new update-incompatible version. Basically freeze the old one and leave it behind (likebeam/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java
Line 87 in 3262018
expand
itself straight-line code except for the top-level decision of which version to expand.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Apologies, but I don't understand 100%.
beam/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
Line 866 in d0def26
beam/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
Line 1080 in d0def26
beam/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
Line 609 in d0def26
So I am adding the reshuffle right before the DoFn that requires stable inputs (in _load_data).
Are you saying to create a new PTransform that wraps TriggerLoadJobs, with expand_2_264_0() that just returns ParDo(TriggerLoadJobs) and expand() returns Reshuffle() | ParDo(TriggerLoadJobs)?