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-7730] Introduce Flink 1.9 Runner #9296

Merged
merged 3 commits into from
Oct 22, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion examples/java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ dependencies {
// https://issues.apache.org/jira/browse/BEAM-3583
// apexRunnerPreCommit project(":runners:apex")
directRunnerPreCommit project(path: ":runners:direct-java", configuration: "shadow")
flinkRunnerPreCommit project(":runners:flink:1.8")
flinkRunnerPreCommit project(":runners:flink:1.9")
// TODO: Make the netty version used configurable, we add netty-all 4.1.17.Final so it appears on the classpath
// before 4.1.8.Final defined by Apache Beam
sparkRunnerPreCommit "io.netty:netty-all:4.1.17.Final"
Expand Down
2 changes: 1 addition & 1 deletion examples/kotlin/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ dependencies {
// https://issues.apache.org/jira/browse/BEAM-3583
// apexRunnerPreCommit project(":runners:apex")
directRunnerPreCommit project(path: ":runners:direct-java", configuration: "shadow")
flinkRunnerPreCommit project(":runners:flink:1.8")
flinkRunnerPreCommit project(":runners:flink:1.9")
// TODO: Make the netty version used configurable, we add netty-all 4.1.17.Final so it appears on the classpath
// before 4.1.8.Final defined by Apache Beam
sparkRunnerPreCommit "io.netty:netty-all:4.1.17.Final"
Expand Down
8 changes: 3 additions & 5 deletions runners/flink/1.7/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,9 @@ def basePath = '..'
project.ext {
// Set the version of all Flink-related dependencies here.
flink_version = '1.7.2'
// Main source directory and Flink version specific code.
main_source_dirs = ["$basePath/src/main/java", "./src/main/java"]
test_source_dirs = ["$basePath/src/test/java", "./src/test/java"]
main_resources_dirs = ["$basePath/src/main/resources"]
test_resources_dirs = ["$basePath/src/test/resources"]
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we keep the ability to override resource dirs?

// Version specific code overrides.
main_source_overrides = ['./src/main/java']
test_source_overrides = ['./src/test/java']
archives_base_name = 'beam-runners-flink-1.7'
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.flink.translation.wrappers.streaming.io;

import org.apache.flink.api.common.functions.StoppableFunction;

/**
* Custom StoppableFunction for backward compatibility.
*
* @see <a
* href="https://github.com/apache/flink/commit/e95b347dda5233f22fb03e408f2aa521ff924996">Flink
* interface removal commit.</a>
*/
public interface BeamStoppableFunction extends StoppableFunction {}
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.flink.streaming;

import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;

/** {@link StreamSource} utilities, that bridge incompatibilities between Flink releases. */
public class StreamSources {

public static <OutT, SrcT extends SourceFunction<OutT>> void run(
StreamSource<OutT, SrcT> streamSource,
Object lockingObject,
StreamStatusMaintainer streamStatusMaintainer,
Output<StreamRecord<OutT>> collector)
throws Exception {
streamSource.run(lockingObject, streamStatusMaintainer, collector);
}
}
8 changes: 3 additions & 5 deletions runners/flink/1.8/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,9 @@ def basePath = '..'
project.ext {
// Set the version of all Flink-related dependencies here.
flink_version = '1.8.2'
// Main source directory and Flink version specific code.
main_source_dirs = ["$basePath/src/main/java", "./src/main/java"]
test_source_dirs = ["$basePath/src/test/java", "./src/test/java"]
main_resources_dirs = ["$basePath/src/main/resources"]
test_resources_dirs = ["$basePath/src/test/resources"]
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we keep the ability to override resource dirs?

// Version specific code overrides.
main_source_overrides = ["${basePath}/1.7/src/main/java", './src/main/java']
test_source_overrides = ["${basePath}/1.7/src/test/java", './src/test/java']
archives_base_name = 'beam-runners-flink-1.8'
}

Expand Down
31 changes: 31 additions & 0 deletions runners/flink/1.9/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* License); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an AS IS BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

def basePath = '..'
/* All properties required for loading the Flink build script */
project.ext {
// Set the version of all Flink-related dependencies here.
flink_version = '1.9.0'
// Version specific code overrides.
main_source_overrides = ["${basePath}/1.7/src/main/java", "${basePath}/1.8/src/main/java", './src/main/java']
test_source_overrides = ["${basePath}/1.7/src/test/java", "${basePath}/1.8/src/test/java", './src/test/java']
Copy link
Contributor

Choose a reason for hiding this comment

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

Not sure if that it easier to reason about the source, but at least we do not have code duplication. So looks fine for now.

archives_base_name = 'beam-runners-flink-1.9'
}

// Load the main build script which contains all build logic.
apply from: "$basePath/flink_runner.gradle"
26 changes: 26 additions & 0 deletions runners/flink/1.9/job-server-container/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* License); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an AS IS BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

def basePath = '../../job-server-container'

project.ext {
resource_path = basePath
}

// Load the main build script which contains all build logic.
apply from: "$basePath/flink_job_server_container.gradle"
31 changes: 31 additions & 0 deletions runners/flink/1.9/job-server/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* License); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an AS IS BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

def basePath = '../../job-server'

project.ext {
// Look for the source code in the parent module
main_source_dirs = ["$basePath/src/main/java"]
test_source_dirs = ["$basePath/src/test/java"]
main_resources_dirs = ["$basePath/src/main/resources"]
test_resources_dirs = ["$basePath/src/test/resources"]
archives_base_name = 'beam-runners-flink-1.9-job-server'
}

// Load the main build script which contains all build logic.
apply from: "$basePath/flink_job_server.gradle"
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.flink.translation.wrappers.streaming.io;

/**
* Custom StoppableFunction for backward compatibility.
*
* @see <a
* href="https://github.com/apache/flink/commit/e95b347dda5233f22fb03e408f2aa521ff924996">Flink
* interface removal commit.</a>
*/
public interface BeamStoppableFunction {

/** Unused method for backward compatibility. */
void stop();
}
dmvk marked this conversation as resolved.
Show resolved Hide resolved
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.flink.streaming;

import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
import org.apache.flink.streaming.runtime.tasks.OperatorChain;
import org.apache.flink.streaming.runtime.tasks.StreamTask;

/** {@link StreamSource} utilities, that bridge incompatibilities between Flink releases. */
public class StreamSources {

public static <OutT, SrcT extends SourceFunction<OutT>> void run(
StreamSource<OutT, SrcT> streamSource,
Object lockingObject,
StreamStatusMaintainer streamStatusMaintainer,
Output<StreamRecord<OutT>> collector)
throws Exception {
streamSource.run(
lockingObject, streamStatusMaintainer, collector, createOperatorChain(streamSource));
}

private static OperatorChain<?, ?> createOperatorChain(AbstractStreamOperator<?> operator) {
return new OperatorChain<>(
operator.getContainingTask(),
StreamTask.createRecordWriters(
operator.getOperatorConfig(), new MockEnvironmentBuilder().build()));
}
}
39 changes: 31 additions & 8 deletions runners/flink/flink_runner.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -41,25 +41,45 @@ description = "Apache Beam :: Runners :: Flink $flink_version"
evaluationDependsOn(":sdks:java:core")
evaluationDependsOn(":runners:core-java")

/*
* Copy & merge source overrides into build directory.
*/
def sourceOverridesBase = "${project.buildDir}/source-overrides/src"

def copySourceOverrides = tasks.register('copySourceOverrides', Copy) {
it.from main_source_overrides
it.into "${sourceOverridesBase}/main/java"
it.duplicatesStrategy DuplicatesStrategy.INCLUDE
}
compileJava.dependsOn copySourceOverrides

def copyTestSourceOverrides = tasks.register('copyTestSourceOverrides', Copy) {
it.from test_source_overrides
it.into "${sourceOverridesBase}/test/java"
it.duplicatesStrategy DuplicatesStrategy.INCLUDE
}
compileTestJava.dependsOn copyTestSourceOverrides

/*
* We have to explicitly set all directories here to make sure each
* version of Flink has the correct overrides set.
*/
def sourceBase = "${project.projectDir}/../src"
sourceSets {
main {
java {
srcDirs = main_source_dirs
srcDirs = ["${sourceBase}/main/java", "${sourceOverridesBase}/main/java"]
}
resources {
srcDirs = main_resources_dirs
srcDirs = ["${sourceBase}/main/resources", "${sourceOverridesBase}/main/resources"]
}
}
test {
java {
srcDirs = test_source_dirs
srcDirs = ["${sourceBase}/test/java", "${sourceOverridesBase}/test/java"]
}
resources {
srcDirs = test_resources_dirs
srcDirs = ["${sourceBase}/test/resources", "${sourceOverridesBase}/test/resources"]
}
}
}
Expand All @@ -71,7 +91,7 @@ sourceSets {
*/
spotless {
java {
target project.sourceSets.main.allJava + project.sourceSets.test.allJava
target target + project.fileTree(project.projectDir.parentFile) { include 'src/*/java/**/*.java' }
}
}

Expand All @@ -86,6 +106,9 @@ test {
// Run them serially for now, to avoid "Exit code 137", i.e. Jenkins host killing the Gradle test process
if (project.path == ":runners:flink:1.8") {
mustRunAfter(":runners:flink:1.7:test")
} else if (project.path == ":runners:flink:1.9") {
mustRunAfter(":runners:flink:1.7:test")
mustRunAfter(":runners:flink:1.8:test")
}
}

Expand Down Expand Up @@ -142,7 +165,7 @@ class ValidatesRunnerConfig {

def createValidatesRunnerTask(Map m) {
def config = m as ValidatesRunnerConfig
tasks.create(name: config.name, type: Test) {
tasks.register(config.name, Test) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why this change?

group = "Verification"
def runnerType = config.streaming ? "streaming" : "batch"
description = "Validates the ${runnerType} runner"
Expand Down Expand Up @@ -178,8 +201,8 @@ def createValidatesRunnerTask(Map m) {
createValidatesRunnerTask(name: "validatesRunnerBatch", streaming: false)
createValidatesRunnerTask(name: "validatesRunnerStreaming", streaming: true)

task validatesRunner {
group = "Verification"
tasks.register('validatesRunner') {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why this change?

Copy link
Member Author

Choose a reason for hiding this comment

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

this is recommended approach since 5.1.x as it creates the task lazily

https://docs.gradle.org/current/userguide/task_configuration_avoidance.html

group = 'Verification'
description "Validates Flink runner"
dependsOn validatesRunnerBatch
dependsOn validatesRunnerStreaming
Expand Down
Loading