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-10612] add Flink 1.11 runner #12564

Merged
merged 1 commit into from Aug 15, 2020
Merged

Conversation

nevillelyh
Copy link
Contributor

@nevillelyh nevillelyh commented Aug 13, 2020

WIP

Almost there except the following error:

> Task :runners:flink:1.11:compileJava
/home/neville/src/apache/beam-git/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java:702: error: timeServiceManager has private access in AbstractStreamOperator
      timeServiceManager.advanceWatermark(new Watermark(inputWatermarkHold));

I tried working around it by forking the test & back porting the StreamOperatorStateContext.internalTimerServiceManager logic from upstream but ran into other issues including checkstyle errors w.r.t. missing package-info.java and test failures.

It worked with my forked upstream Flink build with AbstractStreamOperator.timeServiceManager reverted to protected.

@mxm any ideas?


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.

Post-Commit Tests Status (on master branch)

Lang SDK Dataflow Flink Samza Spark Twister2
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
Build Status
Build Status
Python Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
--- Build Status ---
XLang Build Status --- Build Status --- 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 Build Status
Portable --- Build Status --- ---

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

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels

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

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.

Awesome @nevillelyh! Thanks for the PR. A couple comments:

  1. Please do not duplicate/move any testing code. Whenever necessary, add a version-specific override or Reflection, but keep the tests in the base source folder. We try to minimize the amount of code duplication for the supported Flink versions to a minimum.

  2. For the TimerServiceManager, the field is private but there is a getter! Please use the getter AbstractStreamOperator#getTimeServiceManager().

  3. Please update the docs with information on the newly supported version.

* OptimizerPlanEnvironment has been changed in Flink 1.10, please refer to
* https://github.com/apache/flink/commit/0ea4dd7e9d56a017743ca6794d28537800faab6f for more details.
*/
public class FlinkRunnerTest {
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you think we could avoid duplicating all this code? If the signature changed, we should rather define an Interface to retrieve the pipeline/construct the environment, which different versions can override. Alternatively, using reflection is also a valid option in tests.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah should be doable. I'll add a shim.
Also totally missed getTimeServiceManager. My IDEA is still glitchy with the setup, need to get use to the multiple versions code sharing flow 😂

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 addressed most code copy issues by adding compat layers. ./gradlew -p runners/flink test passes for all versions but build fails checkstyle due to missing package-info.java, even though it exists in the base src/main dir. Any idea how to fix?

I'll look at the docs next. Not sure about the CI test scripts though.

> Task :runners:flink:1.10:checkstyleMain FAILED
[ant:checkstyle] [ERROR] /home/neville/src/apache/beam/runners/flink/1.10/build/source-overrides/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java:1: Missing package-info.java file. [JavadocPackage]

> Task :runners:flink:1.8:checkstyleMain FAILED
[ant:checkstyle] [ERROR] /home/neville/src/apache/beam/runners/flink/1.8/build/source-overrides/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java:1: Missing package-info.java file. [JavadocPackage]

> Task :runners:flink:1.9:checkstyleMain
[ant:checkstyle] [ERROR] /home/neville/src/apache/beam/runners/flink/1.9/build/source-overrides/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java:1: Missing package-info.java file. [JavadocPackage]

> Task :runners:flink:1.9:checkstyleMain FAILED

> Task :runners:flink:1.11:checkstyleMain
[ant:checkstyle] [ERROR] /home/neville/src/apache/beam/runners/flink/1.11/build/source-overrides/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/AbstractStreamOperatorCompat.java:1: Missing package-info.java file. [JavadocPackage]

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks!

If the problem doesn't go away after running ./gradlew -p runners/flink clean, then you might have to add a suppression here:

<suppress checks="JavadocPackage" files=".*runners.flink.*CoderTypeSerializer\.java"/>

@nevillelyh nevillelyh force-pushed the neville/flink-1.11 branch 3 times, most recently from d2f3a75 to 8ee08b5 Compare August 13, 2020 17:58
@@ -17,7 +17,6 @@
*/

def basePath = '..'

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: unrelated change


@Override
public int getClusterPort() {
return getClusterInformation().getBlobServerPort();
Copy link
Contributor

Choose a reason for hiding this comment

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

Is it really the blob server port here or do we want the Job server RPC port?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah I really monkeyed that one 🙈

If I understood it correctly, it's a workaround for this, to expose port in useSingleRpcService mode:
https://github.com/apache/flink/blob/release-1.10.1/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java#L270

Which was changed to createLocalRpcService in 1.11.
https://github.com/apache/flink/blob/release-1.11.1/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java#L271

Pushed a fix for this.

* OptimizerPlanEnvironment has been changed in Flink 1.10, please refer to
* https://github.com/apache/flink/commit/0ea4dd7e9d56a017743ca6794d28537800faab6f for more details.
*/
public class FlinkRunnerTest {
Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks!

If the problem doesn't go away after running ./gradlew -p runners/flink clean, then you might have to add a suppression here:

<suppress checks="JavadocPackage" files=".*runners.flink.*CoderTypeSerializer\.java"/>

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.

LGTM! Great work. Thanks @nevillelyh.

Could you squash the commits?

@@ -699,7 +699,7 @@ public final void processWatermark1(Watermark mark) throws Exception {

long inputWatermarkHold = applyInputWatermarkHold(getEffectiveInputWatermark());
if (keyCoder != null) {
timeServiceManager.advanceWatermark(new Watermark(inputWatermarkHold));
getTimeServiceManagerCompat().advanceWatermark(new Watermark(inputWatermarkHold));
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe add a field for this instead of calling the getter every time?

- Fix deprecated OperatorStateStore.getOperatorState => getListState
- Fix WindowedValue OutputTag in DoFnOperatorTest and ExecutableStageDoFnOperatorTest
- Fix FlinkStreamingTransformTranslatorsTest
- Fix SourceTransformation => LegacySourceTransformation rename
- Fix timeServiceManager access change in AbstractStreamOperator
- Fix RemoteMiniClusterImpl RPC service port work around
- Abstract version specific env and PackagedProgram logic in FlinkRunnerTest
- Suppress checkstyle false alarm on AbstractStreamOperatorCompat
@nevillelyh
Copy link
Contributor Author

Squashed & made timeServiceManager a field. Had to rename it though to avoid conflict with <= 1.10.

@mxm
Copy link
Contributor

mxm commented Aug 14, 2020

Run PythonDocker PreCommit

@mxm
Copy link
Contributor

mxm commented Aug 14, 2020

Run Java_Examples_Dataflow PreCommit

@mxm
Copy link
Contributor

mxm commented Aug 14, 2020

Run Portable_Python PreCommit

@mxm
Copy link
Contributor

mxm commented Aug 14, 2020

Run Python2_PVR_Flink PreCommit

@mxm
Copy link
Contributor

mxm commented Aug 14, 2020

Most builds fail at the moment:

20:14:18 * What went wrong:
20:14:18 Execution failed for task ':sdks:go:buildLinuxAmd64'.
20:14:18 > Build failed due to return code 1 of: 
20:14:18   Command:
20:14:18    /usr/bin/go build -o ./build/bin/beamctl github.com/apache/beam/sdks/go/cmd/beamctl
20:14:18   Env:
20:14:18    GOEXE=
20:14:18    GOPATH=/home/jenkins/jenkins-slave/workspace/beam_PreCommit_Java_Commit@3/src/sdks/go/.gogradle/project_gopath
20:14:18    GOROOT=/usr/lib/go-1.12
20:14:18    GOOS=linux
20:14:18    GOARCH=amd64

Considering you only squashed and it was passing before (https://ci-beam.apache.org/job/beam_PreCommit_Java_Commit/13017/), it would probably be fine, but I'll try to kick off the tests again.

@mxm
Copy link
Contributor

mxm commented Aug 14, 2020

retest this please

1 similar comment
@mxm
Copy link
Contributor

mxm commented Aug 15, 2020

retest this please

@mxm
Copy link
Contributor

mxm commented Aug 15, 2020

Run Python PreCommit

@mxm
Copy link
Contributor

mxm commented Aug 15, 2020

Unrelated test failure in Python:

Test Result (2 failures / +1)

    apache_beam.transforms.ptransform_test.PTransformTypeCheckTestCase.test_pardo_properly_type_checks_using_type_hint_decorators

https://ci-beam.apache.org/job/beam_PreCommit_Python_Commit/14588/

@mxm mxm merged commit efad73c into apache:master Aug 15, 2020
@mxm
Copy link
Contributor

mxm commented Aug 15, 2020

Thanks for making this happen!

@nevillelyh nevillelyh deleted the neville/flink-1.11 branch August 17, 2020 18:41
@entimaniac
Copy link

Hey, I know this is a closed PR but I'm hoping someone can help me. I'm trying to understand how I might be able to take advantage of this PR. The environment I'm trying to run my beam job on has flink 1.11 installed but the only runner package I see available is for 1.10 (https://mvnrepository.com/artifact/org.apache.beam/beam-runners-flink-1.10) Is there another way of utilizing this code?

@mxm
Copy link
Contributor

mxm commented Sep 15, 2020 via email

@entimaniac
Copy link

thanks! that was all the information I was looking to hear

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.

None yet

3 participants