apache / beam

Apache Beam is a unified programming model for Batch and Streaming data processing.
https://beam.apache.org/
Apache License 2.0
7.88k stars 4.27k forks source link

[Bug]: Large PIPELINE_OPTIONS can lead to command line args list too long errors in SDK containers #27839

Open lostluck opened 1 year ago

lostluck commented 1 year ago

What happened?

A Dataflow customer with a large number of --filesToStage leads to workers unable to boot up, failing with Java exited: fork/exec /opt/java/openjdk/bin/java: argument list too long.

After some investigation, it's revealed that in Linux, Environment variables take up command line length apparently:

https://stackoverflow.com/questions/28865473/setting-environment-variable-to-a-large-value-argument-list-too-long

And Beam Java serializes the pipeline options in JSON format to an evironement variable.

https://github.com/apache/beam/blob/release-2.49.0/sdks/java/container/boot.go#L128

This also happens for Python:

https://github.com/apache/beam/blob/90809097260ec4252b746b97bd849efc412950f5/sdks/python/container/boot.go#L206 but no reports for this as of yet.

Previous work to resolve this was here, focused on the Java class path: https://github.com/apache/beam/issues/25582

While that certainly helped the issue, large Pipeline options remain an issue.

The proposed fix for Java at least is to write another environment variable PIPELINE_OPTIONS_FILE, which will contain the file location for a json encoded version of the pipeline options, similar to how we've done the pathing jar.

The behavior from the portable SDK harness should be to look at this environment variable, and if it exists, read the JSON pipeline options from them. Otherwise, fall back to the existing behavior.

This allows for slight mismatch in container versions vs Beam versions for users who aren't experiencing this issue.

Issue Priority

Priority: 1 (data loss / total loss of function)

Issue Components

lostluck commented 1 year ago

https://github.com/apache/beam/blob/90809097260ec4252b746b97bd849efc412950f5/sdks/python/apache_beam/runners/worker/sdk_worker_main.py#L92 is where the Python stuff primarily happens.

and

https://github.com/apache/beam/blob/90809097260ec4252b746b97bd849efc412950f5/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java#L92 is the same for Java.

lostluck commented 1 year ago

I'm unable to gain confidence that this PR doesn't simply break things, as tests weren't failing without the java change, or with the java change.

Since I don't have the ability to do anything quickly with the Java SDK, I'm unable to validate this before the 2.50 cut. However, users could build a custom container in the mean time, based on the beam branch once we do successfully get this in, but are unable to cherry pick it.

lostluck commented 1 year ago

This should be resolved for Java in 2.50+. It's lower priority to do the same for Java and Go, but applying this protocol consistently across SDKs will prevent future issues.

damccorm commented 1 year ago

@lostluck I noticed we've bumped this back several milestones - any reason to keep a milestone on it vs just removing it entirely and letting it get autotagged when we get to it?

Guessing its not actually a 2.52 release blocker

lostluck commented 1 year ago

Agreed.

Java was the only one impacted, but migrating to this approach for all SDKs would be for parity.

lostluck commented 5 months ago

A Dataflow Go SDK user has run into an issue that seems like this approach would resolve, so I'm going to see about completing the change for the Go SDK. A volunteer for the Python SDK would be appreciated, so we can close the issue (though ensuring Typescript and Swift aren't blockers, they may as well move to the paradigm to avoid future issues, and assure consistency among SDKs).