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-5443] Pipeline option defaults for portable runner. #6512

Merged
merged 1 commit into from
Sep 29, 2018

Conversation

tweise
Copy link
Contributor

@tweise tweise commented Sep 27, 2018

Attempt to set portable runner specific defaults for experiments and sdk_location options.


Follow this checklist to help us incorporate your contribution quickly and easily:

  • 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.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

It will help us expedite review of your Pull Request if you tag someone (e.g. @username) to look at it.

Post-Commit Tests Status (on master branch)

Lang SDK Apex Dataflow Flink Gearpump Samza Spark
Go Build Status --- --- --- --- --- ---
Java Build Status Build Status Build Status Build Status Build Status Build Status Build Status
Python Build Status --- Build Status
Build Status
Build Status --- --- ---

@tweise
Copy link
Contributor Author

tweise commented Sep 27, 2018

R: @angoenka

@tweise
Copy link
Contributor Author

tweise commented Sep 27, 2018

The command line with these changes:

python -m apache_beam.examples.wordcount   --input=/etc/profile   --output=/tmp/py-wordcount-direct   --runner=PortableRunner  --job_endpoint=localhost:8099  --streaming

Copy link
Contributor

@angoenka angoenka left a comment

Choose a reason for hiding this comment

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

Thanks!
Just a comment of checking the items in experiments instead of just the experiments.

# (needs to occur prior to pipeline construction)
if self._options.view_as(StandardOptions).runner == 'PortableRunner':
self._options.view_as(DebugOptions).experiments = (
self._options.view_as(DebugOptions).experiments or ['beam_fn_api'])
Copy link
Contributor

Choose a reason for hiding this comment

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

We should check and appendbeam_fn_api to experiments.

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 idea was to only touch experiments when the user did not set it. Do you think we should just always add beam_fn_api instead?

Copy link
Contributor

Choose a reason for hiding this comment

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

I think it should be ok to add beam_fn_api always as without it the pipeline will fail.
And the behavior will be inconsistent where when no experiment is provided then the pipeline works but when when experiments is provided without explicit beam_fn_api, the pipeline fails.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done!

Copy link
Contributor

@angoenka angoenka left a comment

Choose a reason for hiding this comment

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

Thanks!


# portable runner specific default
if pipeline.options.view_as(SetupOptions).sdk_location == 'default':
pipeline.options.view_as(SetupOptions).sdk_location = 'container'
Copy link
Contributor

Choose a reason for hiding this comment

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

I wonder, shouldn't we change the interpretation of "default" in case of the portable runner to be container, instead of changing the value here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That would be cleaner, but stager.py may be used by dataflow or other modules that have a different interpretation of default. @angoenka ?

Copy link
Contributor

Choose a reason for hiding this comment

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

For Dataflow, default means pypi.
Default can mean different thing based on the runner so translating default to container seems reasonable.

Copy link
Contributor

Choose a reason for hiding this comment

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

If we can change the interpretation of default in the FlinkRunner to be container, I'd prefer that. It looks like changing this here could cause unexpected behavior.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It is kind of strange to have that 'default' default value. Should it not be just None when there is no real default? Maybe we can change that in a follow-up 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.

@mxm I don't think it can be deferred to the runner, the stager behavior depends on it. I would prefer that the stager does nothing unless a specific option was set (like pypi in dataflow case), but that should probably be a separate change. It would allow us to not even touch the value in portable runner.

Copy link
Contributor

Choose a reason for hiding this comment

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

You're right, that doesn't work. I just find it odd to rewrite default -> container when the default could be container. The problem is that this option is used in different contexts (Portable Runner / Dataflow Runner).

Couldn't we alter the behavior of stager when the default value is used?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

sdks/python/apache_beam/pipeline.py Show resolved Hide resolved
@tweise tweise force-pushed the BEAM-5443.portableDefaultOptions branch 2 times, most recently from d03388a to 4940b7c Compare September 29, 2018 01:05
@tweise tweise force-pushed the BEAM-5443.portableDefaultOptions branch from 4940b7c to d3e6bab Compare September 29, 2018 20:27
@tweise tweise merged commit 766a1dc into apache:master Sep 29, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants