Skip to content

Conversation

@tgroh
Copy link
Member

@tgroh tgroh commented Apr 11, 2016

Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

  • Make sure the PR title is formatted like:
    [BEAM-<Jira issue #>] Description of pull request
  • Make sure tests pass via mvn clean verify. (Even better, enable
    Travis-CI on your fork and ensure the whole test matrix passes).
  • Replace "<Jira issue #>" in the title with the actual Jira issue
    number, if there is one.
  • If this contribution is large, please file an Apache
    Individual Contributor License Agreement.

The SparkPipelineOptionsFactory should ensure that the returned options
have Spark as the PipelineRunner.

Use the SparkPipelineOptionsFactory in the spark TfIdf test

Without using the SparkPipelineRunner explicitly, the Pipeline run with
the SparkRunner may have an unexpected graph due to runner-specific
interceptions of the Pipeline#apply method

@tgroh
Copy link
Member Author

tgroh commented Apr 11, 2016

R: @amitsela

@lukecwik
Copy link
Member

@amitsela:
Why do we have SparkPipelineOptionsFactory?
Any reason not to just use PipelineOptionsFactory.as(SparkPipelineOptions.class)?

@amitsela
Copy link
Member

@lukecwik I guess SparkPipelineOptionsFactory.create() was a wrapper for PipelineOptionsFactory.as(SparkPipelineOptions.class).

@tgroh It makes sense that if we provide a PipelineOptionsFactory.as wrapper to the Spark runner we should apply the SparkRunner there, but we should do the same in SparkStreamingPipelineOptionsFactory.create()

So it's either letting the user choose the correct options as @lukecwik suggests:
PipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class) OR PipelineOptionsFactory.as(SparkStreamingPipelineOptions.class)
followed by options.setRunner(SparkPipelineRunner.class)

Or as @tgroh suggests:
SparkPipelineOptions opts = SparkPipelineOptionsFactory.create() OR SparkStreamingPipelineOptionsFactory.create()

As the model writers, what is the correct way for a runner implementor to choose ? Which way we want the user to use this ? should he explicitly state the Runner or choosing the options will provide the correct runner ?

@tgroh
Copy link
Member Author

tgroh commented Apr 12, 2016

Generally Pipeline authors should not have to use any runner-specific classes during pipeline construction or submission; so users should call neither PipelineOptionsFactory.as(SparkPipelineOptions.class) nor SparkPipelineOptionsFactory.create(), and values for runner-specific options can be provided via PipelineOptionsFactory.fromArgs(String[]), with the actual runner implementation responsible for ensuring that the provided options are valid (via obtaining the appropriate subtype).

Pipeline authors are responsible for selecting a runner, currently at or before the time of Pipeline.create(), and at the time of Pipeline.run() as we move to the new runner API - they should do so, however, by using PipelineOptions.setRunner(Class<? extends PipelineRunner>) or passing the appropriate runner name in PipelineOptionsFactory.fromArgs(String[])

@lukecwik
Copy link
Member

+1 for Thomas Groh response

On Tue, Apr 12, 2016 at 8:54 AM, Thomas Groh notifications@github.com
wrote:

Generally Pipeline authors should not have to use any runner-specific
classes during pipeline construction or submission; so users should call
neither PipelineOptionsFactory.as(SparkPipelineOptions.class) nor
SparkPipelineOptionsFactory.create(), and values for runner-specific
options can be provided via PipelineOptionsFactory.fromArgs(String[]),
with the actual runner implementation responsible for ensuring that the
provided options are valid (via obtaining the appropriate subtype).

Pipeline authors are responsible for selecting a runner, currently at or
before the time of Pipeline.create(), and at the time of Pipeline.run()
as we move to the new runner API - they should do so, however, by using PipelineOptions.setRunner(Class<?
extends PipelineRunner>) or passing the appropriate runner name in
PipelineOptionsFactory.fromArgs(String[])


You are receiving this because you were mentioned.
Reply to this email directly or view it on GitHub
#167 (comment)

@amitsela
Copy link
Member

I can see your point, and it's true for PipelineOptionsFactory.fromArgs(String[]).
I can still see why users who work mainly with one runner would benefit from a dedicated PipelineOptionsFactory, but in that case it should also set the runner (as done here by @tgroh )

so I guess it's +1 for me as well

@tgroh
Copy link
Member Author

tgroh commented Apr 12, 2016

I've pushed a change to remove the factories, and fix up all of the tests in which they were used.

If this commit is merged, the first in this PR should be discarded.

@tgroh tgroh changed the title Select the runner in spark SparkPipelineOptionsFactory Remove SparkPipelineOptionsFactory, SparkStreamingPipelineOptionsFactory Apr 13, 2016
@kennknowles
Copy link
Member

@tgroh I don't quite understand your last comment. Perhaps you could just rebase to exactly what you propose to merge?

@tgroh
Copy link
Member Author

tgroh commented Apr 13, 2016

@kennknowles done.

@tgroh tgroh force-pushed the spark_options_runner_choice branch from 1fbe48e to 2f5be67 Compare April 13, 2016 16:10
@amitsela
Copy link
Member

+1 for removing the factories as well

@tgroh tgroh force-pushed the spark_options_runner_choice branch from 2f5be67 to 95472e0 Compare April 14, 2016 16:39
Pipeline authors should generally not use any runner-specific classes,
but instead should select the runner and appropriate configurations
through the PipelineOptionsFactory.fromArgs() method. The runner can
then obtain the appropriately typed PipelineOptions class as required
and do any neccessary validation. Failing this, they should use the
provided PipelineOptions#as() method to acquire the appropriately typed
options.

If required, users should construct SparkPipelineOptions via
PipelineOptionsFactory.as(SparkPipelineOptions.class).
@tgroh tgroh force-pushed the spark_options_runner_choice branch from 95472e0 to 11ba2b9 Compare April 14, 2016 17:36
@tgroh
Copy link
Member Author

tgroh commented Apr 14, 2016

Rebased on top of beam. If there aren't any more comments, this is ready to merge, pending jenkins

@lukecwik
Copy link
Member

LGTM

@asfgit asfgit merged commit 11ba2b9 into apache:master Apr 14, 2016
asfgit pushed a commit that referenced this pull request Apr 14, 2016
swegner pushed a commit to swegner/beam that referenced this pull request Apr 22, 2016
iemejia pushed a commit to iemejia/beam that referenced this pull request Jan 12, 2018
damccorm added a commit that referenced this pull request Dec 9, 2022
…f-hosted runners (#23134)

* Updating build_playground_backend workflow (#167)

Co-authored-by: Elias Segundo <elias.segundo@luisrazo.local>

* Added master changes in build_playground_backend to avoid merge conflicts

* Reverted GO_VERSION and BEAM_VERSION to have the same as master

build_playground_backend

* Switching trigger to pull_request (#259)

* Switching to pull_request

* Removing ref from checkout

Co-authored-by: Elias Segundo Antonio <eliassegundo.segundo@gmail.com>
Co-authored-by: Elias Segundo <elias.segundo@luisrazo.local>
Co-authored-by: elink22 <103056145+elink22@users.noreply.github.com>
Co-authored-by: Danny McCormick <dannymccormick@google.com>
lostluck pushed a commit to lostluck/beam that referenced this pull request Dec 22, 2022
…f-hosted runners (apache#23134)

* Updating build_playground_backend workflow (apache#167)

Co-authored-by: Elias Segundo <elias.segundo@luisrazo.local>

* Added master changes in build_playground_backend to avoid merge conflicts

* Reverted GO_VERSION and BEAM_VERSION to have the same as master

build_playground_backend

* Switching trigger to pull_request (apache#259)

* Switching to pull_request

* Removing ref from checkout

Co-authored-by: Elias Segundo Antonio <eliassegundo.segundo@gmail.com>
Co-authored-by: Elias Segundo <elias.segundo@luisrazo.local>
Co-authored-by: elink22 <103056145+elink22@users.noreply.github.com>
Co-authored-by: Danny McCormick <dannymccormick@google.com>
pl04351820 pushed a commit to pl04351820/beam that referenced this pull request Dec 20, 2023
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.

5 participants