Skip to content

Conversation

@tweise
Copy link
Contributor

@tweise tweise commented Oct 24, 2019

Add a new Flink entry point (main method) that invokes the external SDK client entry point to generate the pipeline and submits the resulting Flink job like any other Flink native driver program would, via the optimizer plan environment ("[auto]").

Note that in this PR, the SDK client is assumed to be on the same host, which is the case when Flink and Python dependencies are in the same container image, for example. While this is something that can be solved at build time, the question from Robert made me realize that the dependency is almost identical to that between the runner and SDK on the execution side, which is abstracted via the environment concept. So we could, in the future, consider introducing a "client environment" or simply expand the existing environment. This would allow the SDK bits for both pipeline construction and execution to live in a side car container, separate from Flink.


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

Post-Commit Tests Status (on master branch)

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

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable Build Status Build Status
Build Status
Build Status Build Status
Portable --- Build Status --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

@tweise
Copy link
Contributor Author

tweise commented Oct 24, 2019

R: @chadrik

@chadrik
Copy link
Contributor

chadrik commented Oct 24, 2019

Thanks for the heads up. I don't have any input on this, but I do like staying in the loop on this subject!

Copy link
Contributor

@mxm mxm left a comment

Choose a reason for hiding this comment

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

Thanks @tweise. Looks good! A couple of minor comments and clarifications inline.

Copy link
Contributor

Choose a reason for hiding this comment

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

We may want to make this executable configurable, for non bash users. How about putting executable and args in the config with the current defaults?

Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
this.driverCmd = driverCmd;
Preconditions.checkState(!driverCmd.contains(DRIVER_CMD_FLAGS), "Driver command must not contain "+ DRIVER_CMD_FLAGS);
this.driverCmd = driverCmd;

Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
private static final String DRIVER_CMD_FLAGS = "--job_endpoint=%s";
private static final String JOB_ENDPOINT_FLAG = "--job_endpoint";

Copy link
Contributor

Choose a reason for hiding this comment

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

This allows us to check for the args in the user command string. The %s would have to be appended in the format string.

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 any preference regarding class name? I first had it as FlinkPortableClientRunner but maybe "runner" is misleading. On the other hand, "entry point" is also redundant.

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 just FlinkPortableRunner? The functionality is closest to the classic FlinkRunner, as it allows for any portable pipeline to be submitted.

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 had it as XYZRunner originally, but figured it will cause confusion. There are FlinkPipelineRunner and PortablePipelineRunner already (these are used within the job server).

Copy link
Contributor

Choose a reason for hiding this comment

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

We could rename those 🤓

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 just FlinkPortableRunner? The functionality is closest to the classic FlinkRunner, as it allows for any portable pipeline to be submitted.

Copy link
Contributor

Choose a reason for hiding this comment

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

This comment is obsolete, no?

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, updated now.

Copy link
Contributor Author

@tweise tweise Oct 28, 2019

Choose a reason for hiding this comment

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

@mxm I see the inherit working as the expected output is printed in the console. But short of starting a separate java process I don't see a way to capture it in the test.

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 left the test but skipped the assertion, maybe we can still find a clever way to do this.

@tweise tweise force-pushed the BEAM-8471.flinkPortableClientRunner branch from 3b5dce8 to f89c1b9 Compare October 28, 2019 20:02
tweise added a commit to lyft/beam that referenced this pull request Oct 28, 2019
@tweise tweise merged commit 0645404 into apache:master Oct 28, 2019
@tweise tweise deleted the BEAM-8471.flinkPortableClientRunner branch October 28, 2019 22:27
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