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

[BEAM-14038] Auto-startup for Python expansion service. #17035

Merged
merged 12 commits into from
Mar 17, 2022

Conversation

robertwb
Copy link
Contributor

@robertwb robertwb commented Mar 7, 2022

This adds a script and some java code for automatically starting a Beam Python service.

It may be useful for Go, Typescript, etc. as well.


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

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
  • 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

See CI.md for more information about GitHub Actions CI.

@robertwb
Copy link
Contributor Author

robertwb commented Mar 7, 2022

R: @ihji

@robertwb robertwb changed the title Auto-startup for Python expansion service. [BEAM-14038] Auto-startup for Python expansion service. Mar 7, 2022
@codecov
Copy link

codecov bot commented Mar 7, 2022

Codecov Report

Merging #17035 (04b843d) into master (2aa4da0) will increase coverage by 0.07%.
The diff coverage is n/a.

@@            Coverage Diff             @@
##           master   #17035      +/-   ##
==========================================
+ Coverage   73.81%   73.89%   +0.07%     
==========================================
  Files         667      667              
  Lines       87340    88017     +677     
==========================================
+ Hits        64474    65039     +565     
- Misses      21763    21875     +112     
  Partials     1103     1103              
Flag Coverage Δ
python 83.63% <ø> (-0.01%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
...nternal/clients/cloudbuild/cloudbuild_v1_client.py 52.21% <0.00%> (-2.97%) ⬇️
..._beam/runners/portability/sdk_container_builder.py 37.93% <0.00%> (-1.02%) ⬇️
...che_beam/runners/interactive/interactive_runner.py 92.02% <0.00%> (-0.73%) ⬇️
...dks/python/apache_beam/options/pipeline_options.py 94.85% <0.00%> (-0.43%) ⬇️
sdks/python/apache_beam/coders/coder_impl.py 93.97% <0.00%> (-0.31%) ⬇️
...hon/apache_beam/runners/worker/bundle_processor.py 93.39% <0.00%> (-0.25%) ⬇️
...eam/runners/interactive/interactive_environment.py 90.51% <0.00%> (-0.23%) ⬇️
...ython/apache_beam/io/gcp/experimental/spannerio.py 82.15% <0.00%> (-0.17%) ⬇️
sdks/python/apache_beam/transforms/util.py 95.96% <0.00%> (-0.17%) ⬇️
...ks/python/apache_beam/runners/worker/sdk_worker.py 88.90% <0.00%> (-0.16%) ⬇️
... and 13 more

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 2aa4da0...04b843d. Read the comment docs.

@robertwb robertwb force-pushed the py-expansion-service branch from b2e8bff to b6f214e Compare March 8, 2022 23:28
@robertwb robertwb force-pushed the py-expansion-service branch from b6f214e to bc7db3e Compare March 8, 2022 23:37
@robertwb
Copy link
Contributor Author

robertwb commented Mar 9, 2022

Failure in org.apache.beam.sdk.transforms.ParDoLifecycleTest.testTeardownCalledAfterExceptionInFinishBundleStateful

@robertwb
Copy link
Contributor Author

robertwb commented Mar 9, 2022

Run Java PreCommit

@robertwb
Copy link
Contributor Author

robertwb commented Mar 9, 2022

Run CommunityMetrics PreCommit

@robertwb
Copy link
Contributor Author

Flake due to BEAM-8101

@robertwb
Copy link
Contributor Author

Run Java PreCommit

Copy link
Contributor

@ihji ihji left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, looks good.

.withMultiOutputs();
PCollectionTuple outputs;
if (input instanceof PCollection) {
outputs = (PCollectionTuple) ((PCollection) input).apply(transform);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need this explicit casting to (PCollectionTuple) ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, unfortunately. It seems java struggles with "paritally-raw" types.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, I was able to work around this by using PCollection<?>.

List<String> bootstrapCommand = ImmutableList.of("python", bootstrapScript.getAbsolutePath());
LOG.info("Running bootstrap command " + bootstrapCommand);
Process bootstrap =
new ProcessBuilder("python", bootstrapScript.getAbsolutePath())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't it simpler to run bootstrap_beam_venv.py not by creating a temporary copy but just passing the script text resource to python executable via stdin pipe?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good question, I actually tried to do it this way originally. The problem is that the os.execlp bootstrap from Python 2 to Python 3 doesn't work if it isn't run as a file (as the stdin is already partially consumed). There are still too many systems out there that, despite having Python 3, map python to Python 2.x. (An alternative would be to make the full script Python 2 compatible, but that would cause ongoing pain, especially if one can't count on any dependencies like six...)

parser.add_argument('--python_version', help="Python major version.")
parser.add_argument('--beam_version',
help="Beam version.",
default="2.36.0")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there any way to avoid hard-coded Beam version? Do we need to update this version string every time we release a new version of Beam?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could consider making this a required argument. (Note that it's not an error to mix and match Beam versions, as long as what you're counting on is available, though it's generally best to have the latest.

Schema.Field.of("constructor", Schema.FieldType.STRING),
Schema.Field.of("args", Schema.FieldType.row(args.getSchema())),
Schema.Field.of("kwargs", Schema.FieldType.row(kwargs.getSchema())));
payloadSchema.setUUID(UUID.randomUUID());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just out of curiosity, why do we need to set UUID for the schema?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Python has a schema cache (e.g. so it's not constantly re-creating types for a dynamic schema). Results in some very surprising behavior if this is not set.

@robertwb robertwb force-pushed the py-expansion-service branch from 1326808 to 731b9a3 Compare March 11, 2022 06:17
@robertwb robertwb force-pushed the py-expansion-service branch from 731b9a3 to 049408c Compare March 14, 2022 16:22
Copy link
Contributor

@chamikaramj chamikaramj left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks.

import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;

/** Wrapper for invoking external Python transforms. */
public class ExternalPythonTransform<InputT extends PInput, OutputT extends POutput>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We want this to be the main API for Java cross-language transforms, right ? So probably we should advertise in the PR description.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

See below.

private final Row args;
private final Row kwargs;

public ExternalPythonTransform(String fullyQualifiedName, Row args, Row kwargs) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about adding following that wraps this transform:
External.of (String transformName, Map parameters, String endpoint);

And may be deprecate existing:
External.of(String urn, byte[] payload, String endpoint)

(this can be a separate PR)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree that we need a nicer API, which should probably have a 1-pager and be a separate PR. (To seed this, we should have a RowBuilder that starts with the empty row and can be built with withField(name, value) with several overloads for different value types that builds up the schema dynamically. I don't know that a Map<String, Object> parameters is the most java-natural, but it could be allowed as an option as well.) I'm not convinced it makes sense to add a back-reference from External to ExternalPython.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Adding a Row builder (on top of this) seems relatively straightforward. See #17101

I think it will be confusing to add another main External transform API (in addition to External.of and ExternalPythonTransform) so probably ExternalPythonTransform should be the main API to be used along with the Row builder.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ExternalPythonTransform only makes sense for Python transforms, as it starts up a Python expansion service (rather than require one be provided) and expects a fqn + positional arg + kwargs rather than a urn + row (though one could argue row is basically kwargs).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, sorry, I meant "main API for using Python transforms from Java".

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

.apply(
new ExternalPythonTransform<
PCollection<KV<String, String>>, PCollection<KV<String, Iterable<String>>>>(
"apache_beam.GroupByKey", Row.nullRow(Schema.of()), Row.nullRow(Schema.of())))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add a more comprehensive set of unit tests here that uses DirectRunner ? Also "apache_beam.GroupByKey" is not a good example since it doesn't involve the SDK during execution.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The Java DirectRunner can't run cross-language pipelines. I used GBK because I wanted something simple and obvious--eventually we'll hopefully have a richer set of transforms to invoke.

@robertwb
Copy link
Contributor Author

Timeout waiting to lock Build cache (/home/jenkins/.gradle/caches/build-cache-1). It is currently in use by another Gradle instance.

@robertwb
Copy link
Contributor Author

Run Java PreCommit

1 similar comment
@robertwb
Copy link
Contributor Author

Run Java PreCommit

@robertwb
Copy link
Contributor Author

Run Java PreCommit

@robertwb
Copy link
Contributor Author

Run Java PreCommit

@robertwb
Copy link
Contributor Author

Failure in org.apache.beam.sdk.io.hadoop.format.HadoopFormatIOElasticTest.classMethod

@robertwb robertwb merged commit 0510cec into apache:master Mar 17, 2022
@lukecwik
Copy link
Member

lukecwik commented Mar 22, 2022

lukecwik added a commit that referenced this pull request Mar 22, 2022
lukecwik added a commit that referenced this pull request Mar 22, 2022
robertwb added a commit to robertwb/incubator-beam that referenced this pull request Mar 31, 2022
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.

4 participants