Skip to content
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

[flink-runner] Improve Datastream for batch performances #32440

Open
wants to merge 32 commits into
base: master
Choose a base branch
from

Conversation

jto
Copy link
Contributor

@jto jto commented Sep 12, 2024

Context

Flink will drop support for the dataset API in 2.0 which should be released by EOY so it quite important for Beam to support Datastream well.

The PR

This PR improves the performances of Batch jobs executed with --useDatastreamForBatch by porting the following performance optimizations already present in FlinkBatchTransformTranslators but lacking in FlinkStreamingTransformTranslators.

It also implements the following optimizations:

  • Use a "lazy" split enumerator to distributes split dynamically rather than eagerly. This new enumerator greatly reduces skew as each slot is able to pull new splits to consume only when it has finished its work.
  • Set the default maxParallelism to parallelism as the total number of key groups is equal to maxParallelism. Again this reduces skew.
  • Make ToKeyedWorkItem part of DoFnOperator which reduces the size of the job graph and avoid unnecessary inter-task communication.
  • Force a common slot-sharing group on every bounded IOs. This emulate the behavior of the Dataset API which again improves performances especially when data is being shuffled several times while partitioning keys are unchanged (for example of the job does GBK -> map -> CombinePerKey). Add a flag to control this feature (defaults to active).
  • Use a custom class for keys which guarantee a good distribution of data even when the number of keys is not >> parallelism.
  • Other minor optimizations removing repeated serde work.

Benchmarks

The patched version was tested against a few of Spotify's production batch workflows. All settings were left unchanged except for the followings:

  • passed --useDatastreamForBatch=true
  • set jobmanager.scheduler: default (otherwise datastream default to adaptive scheduler).
Beam 2.56 - dataset Beam 2.56 - datastream Beam 2.56 - datastream patched
job # workers execution time execution time % diff execution time % diff
Job 1 350 2:19:00 fails after 4h29min - 1:43:00 -25.90%
Job 2 160 0:23:00 0:35:00 52.17% 0:22:36 -1.74%
Job 3 200 0:53:08 1:34:39 78.14% failed -
Job 4 160 2:31:20 4:27:00 76.43% 2:19:35 -7.76%
Job 5 1 0:43:00 not tested - 0:38:00 -11.63%
Job 6 300 3:01:00 not tested - 2:55:00

Note

Job 3 fails with a stackoverflow exception because of a bug in versions of Kryo < 3.0. I believe this is because the job uses taskmanager.runtime.large-record-handler: true and it should be fixed in Flink 2.0 since Kryo is upgraded to a more recent version.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

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

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

@jto jto changed the title Julient/batch datastream Improve Datastream for batch performances Sep 12, 2024
@jto jto force-pushed the julient/batch-datastream branch 3 times, most recently from fea7323 to 893c19f Compare September 13, 2024 09:31
@jto jto changed the title Improve Datastream for batch performances [flink-runner] Improve Datastream for batch performances Sep 13, 2024
@jto jto marked this pull request as ready for review September 13, 2024 10:24
Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @damccorm for label website.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@damccorm
Copy link
Contributor

@kennknowles would you mind taking a look at this one?

Copy link
Contributor

Reminder, please take a look at this pr: @damccorm

Copy link
Contributor

Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:

R: @melap for label website.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

@damccorm
Copy link
Contributor

damccorm commented Oct 3, 2024

R: @kennknowles

Copy link
Contributor

github-actions bot commented Oct 3, 2024

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control. If you'd like to restart, comment assign set of reviewers

@kennknowles
Copy link
Member

To test this thoroughly, let us add some of the postcommits by touching trigger files. In #32648 you can see how I edited the JSON files (including some new ones) and I think these are all the Flink-specific postcommit jobs.

@github-actions github-actions bot added the build label Oct 16, 2024
@jto jto force-pushed the julient/batch-datastream branch 2 times, most recently from e060213 to ae0f3b4 Compare October 17, 2024 14:00
@jto
Copy link
Contributor Author

jto commented Oct 17, 2024

@kennknowles done. I also rebased master but some of the tests seem to be quite flaky now. There are test failing on things I did not touch (direct runner) and the Flink tests that are failing here are not failing on my machine... Any idea how I could make them work ?

@kennknowles
Copy link
Member

I opened jto#236 with some more trigger files. The "PVR" trigger files stands for "Portable Validates Runner" that isn't as directly impacted. I think the non-portable ValidatesRunner tests should test that the runner still complies with the model and passes the basic tests.

@jto
Copy link
Contributor Author

jto commented Nov 5, 2024

Thanks! I just merged it.
Sorry for the slow response. I was on vacation :)

@github-actions github-actions bot added the java label Nov 6, 2024
@jto jto force-pushed the julient/batch-datastream branch 2 times, most recently from cfa9acf to 687113a Compare November 6, 2024 15:26
@github-actions github-actions bot removed the java label Nov 6, 2024
@jto
Copy link
Contributor Author

jto commented Nov 7, 2024

Hey there!

I rebased master into my branch and a few tests are failing however:

In beam_PreCommit_Java (Run Java PreCommit)

  • tests are failing in :runners:spark:3:test. I did not change anything in the spark runner and according to the report those tests are also failing in other builds. I guess my PR is not the cause of those failures.
  • One of the tests in :runners:flink:1.17:test is failing but it succeeds on my machine. It looks like something may not be deterministic. I already fixed a couple of determinism issues in tests. I'll try to fix this one too.

In beam_PostCommit_Java_ValidatesRunner_Flink (Run Flink ValidatesRunner)

  • One test in :runners:flink:1.19:validatesRunnerBatchWithDataStream is failing. This one is a "real" failure. I'll investigate it too.

PostCommit Go VR Flink / beam_PostCommit_Go_VR_Flink (Run Go Flink ValidatesRunner)

Logs are truncated. I don't know if there's an actual failure or what it might be...

@jto
Copy link
Contributor Author

jto commented Nov 18, 2024

Hey @kennknowles!
I tried really hard to make the failing test (org.apache.beam.sdk.transforms.ParDoTest$LifecycleTests) work but in the end I think it's impossible.
The behaviour of the runner is correct (it fails when it's supposed to) but Flink will either fail with the expected exception or just fail with a generic error (TaskNotRunningException: Task is not running, but in state FAILED).
The expected error is there in the logs but the "final" error may or may not be the expected one. The behaviour is non-deterministic.
In the end I just added this class into sickbay and that seems to have fixed almost all the problems.

The Python PostCommits are failing but the error is:

2024-11-18T11:39:33.3464029Z Execution failed for task ':sdks:python:container:py312:docker'.                                                                                                              │
│2024-11-18T11:39:33.3464463Z > Process 'command 'docker'' finished with non-zero exit value 1

which I think is unrelated to those changes and I could not find anything in the logs suggesting otherwise.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants