Skip to content

Commit

Permalink
Merge pull request #14719: [BEAM-12277] Add Flink 1.13 build target.
Browse files Browse the repository at this point in the history
  • Loading branch information
iemejia authored May 28, 2021
2 parents 2516cac + 881da20 commit 00eb420
Show file tree
Hide file tree
Showing 12 changed files with 220 additions and 31 deletions.
2 changes: 1 addition & 1 deletion .test-infra/jenkins/CommonTestProperties.groovy
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ class CommonTestProperties {
}

static String getFlinkVersion() {
return "1.12"
return "1.13"
}

enum Runner {
Expand Down
1 change: 1 addition & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@
## New Features / Improvements

* `CREATE FUNCTION` DDL statement added to Calcite SQL syntax. `JAR` and `AGGREGATE` are now reserved keywords. ([BEAM-12339](https://issues.apache.org/jira/browse/BEAM-12339)).
* Flink 1.13 is now supported by the Flink runner ([BEAM-12277](https://issues.apache.org/jira/browse/BEAM-12277)).
* X feature added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).

## Breaking Changes
Expand Down
2 changes: 1 addition & 1 deletion gradle.properties
Original file line number Diff line number Diff line change
Expand Up @@ -32,5 +32,5 @@ javaVersion=1.8
docker_image_default_repo_root=apache
docker_image_default_repo_prefix=beam_

flink_versions=1.10,1.11,1.12
flink_versions=1.10,1.11,1.12,1.13

33 changes: 33 additions & 0 deletions runners/flink/1.13/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.13.0'
// Version specific code overrides.
main_source_overrides = ["${basePath}/1.10/src/main/java", "${basePath}/1.11/src/main/java", "${basePath}/1.12/src/main/java", './src/main/java']
test_source_overrides = ["${basePath}/1.10/src/test/java", "${basePath}/1.11/src/test/java", "${basePath}/1.12/src/test/java", './src/test/java']
main_resources_overrides = []
test_resources_overrides = []
archives_base_name = 'beam-runners-flink-1.13'
}

// 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.13/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.13/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.13-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,69 @@
/*
* 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.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
import org.apache.flink.api.dag.Transformation;
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.api.transformations.OneInputTransformation;
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.
*
* <p>This change is becauses RecordWriter is wrapped in RecordWriterDelegate in 1.10, please refer
* to https://github.com/apache/flink/commit/2c8b4ef572f05bf4740b7e204af1e5e709cd945c for more
* details.
*/
public class StreamSources {

/**
* Backward compatibility helper for {@link OneInputTransformation} `getInput` method, that has
* been removed in Flink 1.12.
*
* @param source Source to get single input from.
* @return Input transformation.
*/
public static Transformation<?> getOnlyInput(OneInputTransformation<?, ?> source) {
return Iterables.getOnlyElement(source.getInputs());
}

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.createRecordWriterDelegate(
operator.getOperatorConfig(), new MockEnvironmentBuilder().build()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,13 @@ public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup
final String name = group.getMetricIdentifier(metricName, this);
super.notifyOfRemovedMetric(metric, metricName, group);
synchronized (this) {
ps.printf("%s: %s%n", name, Metrics.toString(metric));
try {
ps.printf("%s: %s%n", name, Metrics.toString(metric));
} catch (NullPointerException e) {
// Workaround to avoid a NPE on Flink's DeclarativeSlotManager during unregister
// TODO Remove once FLINK-22646 is fixed on upstream Flink.
log.warn("unable to log details on metric {}", name);
}
}
}

Expand Down
2 changes: 1 addition & 1 deletion sdks/python/apache_beam/options/pipeline_options.py
Original file line number Diff line number Diff line change
Expand Up @@ -1274,7 +1274,7 @@ def _add_argparse_args(cls, parser):
class FlinkRunnerOptions(PipelineOptions):

# These should stay in sync with gradle.properties.
PUBLISHED_FLINK_VERSIONS = ['1.8', '1.9', '1.10', '1.11', '1.12']
PUBLISHED_FLINK_VERSIONS = ['1.10', '1.11', '1.12', '1.13']

@classmethod
def _add_argparse_args(cls, parser):
Expand Down
49 changes: 25 additions & 24 deletions sdks/python/apache_beam/runners/portability/flink_runner_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -337,46 +337,47 @@ def process(self, kv, state=beam.DoFn.StateParam(state_spec)):
if options.view_as(StandardOptions).streaming:
lines_expected.update([
# Gauges for the last finished bundle
'stateful.beam.metric:statecache:capacity: 123',
'stateful.beam.metric:statecache:size: 10',
'stateful.beam.metric:statecache:get: 20',
'stateful.beam.metric:statecache:miss: 0',
'stateful.beam.metric:statecache:hit: 20',
'stateful.beam.metric:statecache:put: 0',
'stateful.beam.metric:statecache:evict: 0',
'stateful.beam_metric:statecache:capacity: 123',
'stateful.beam_metric:statecache:size: 10',
'stateful.beam_metric:statecache:get: 20',
'stateful.beam_metric:statecache:miss: 0',
'stateful.beam_metric:statecache:hit: 20',
'stateful.beam_metric:statecache:put: 0',
'stateful.beam_metric:statecache:evict: 0',
# Counters
'stateful.beam.metric:statecache:get_total: 220',
'stateful.beam.metric:statecache:miss_total: 10',
'stateful.beam.metric:statecache:hit_total: 210',
'stateful.beam.metric:statecache:put_total: 10',
'stateful.beam.metric:statecache:evict_total: 0',
'stateful.beam_metric:statecache:get_total: 220',
'stateful.beam_metric:statecache:miss_total: 10',
'stateful.beam_metric:statecache:hit_total: 210',
'stateful.beam_metric:statecache:put_total: 10',
'stateful.beam_metric:statecache:evict_total: 0',
])
else:
# Batch has a different processing model. All values for
# a key are processed at once.
lines_expected.update([
# Gauges
'stateful).beam.metric:statecache:capacity: 123',
'stateful).beam_metric:statecache:capacity: 123',
# For the first key, the cache token will not be set yet.
# It's lazily initialized after first access in StateRequestHandlers
'stateful).beam.metric:statecache:size: 10',
'stateful).beam_metric:statecache:size: 10',
# We have 11 here because there are 110 / 10 elements per key
'stateful).beam.metric:statecache:get: 12',
'stateful).beam.metric:statecache:miss: 1',
'stateful).beam.metric:statecache:hit: 11',
'stateful).beam_metric:statecache:get: 12',
'stateful).beam_metric:statecache:miss: 1',
'stateful).beam_metric:statecache:hit: 11',
# State is flushed back once per key
'stateful).beam.metric:statecache:put: 1',
'stateful).beam.metric:statecache:evict: 0',
'stateful).beam_metric:statecache:put: 1',
'stateful).beam_metric:statecache:evict: 0',
# Counters
'stateful).beam.metric:statecache:get_total: 120',
'stateful).beam.metric:statecache:miss_total: 10',
'stateful).beam.metric:statecache:hit_total: 110',
'stateful).beam.metric:statecache:put_total: 10',
'stateful).beam.metric:statecache:evict_total: 0',
'stateful).beam_metric:statecache:get_total: 120',
'stateful).beam_metric:statecache:miss_total: 10',
'stateful).beam_metric:statecache:hit_total: 110',
'stateful).beam_metric:statecache:put_total: 10',
'stateful).beam_metric:statecache:evict_total: 0',
])
lines_actual = set()
with open(self.test_metrics_path, 'r') as f:
for line in f:
print(line, end='')
for metric_str in lines_expected:
metric_name = metric_str.split()[0]
if metric_str in line:
Expand Down
4 changes: 4 additions & 0 deletions settings.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,10 @@ include(":runners:flink:1.11:job-server-container")
include(":runners:flink:1.12")
include(":runners:flink:1.12:job-server")
include(":runners:flink:1.12:job-server-container")
// Flink 1.13
include(":runners:flink:1.13")
include(":runners:flink:1.13:job-server")
include(":runners:flink:1.13:job-server-container")
/* End Flink Runner related settings */
include(":runners:twister2")
include(":runners:google-cloud-dataflow-java")
Expand Down
24 changes: 21 additions & 3 deletions website/www/site/content/en/documentation/runners/flink.md
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,7 @@ Starting with Beam 2.18.0, pre-built Flink Job Service Docker images are availab
[Flink 1.10](https://hub.docker.com/r/apache/beam_flink1.10_job_server),
[Flink 1.11](https://hub.docker.com/r/apache/beam_flink1.11_job_server),
[Flink 1.12](https://hub.docker.com/r/apache/beam_flink1.12_job_server).
[Flink 1.13](https://hub.docker.com/r/apache/beam_flink1.13_job_server).
{{< /paragraph >}}

<!-- TODO(BEAM-10214): Use actual lists here and below. -->
Expand Down Expand Up @@ -311,8 +312,8 @@ reference.
## Flink Version Compatibility

The Flink cluster version has to match the minor version used by the FlinkRunner.
The minor version is the first two numbers in the version string, e.g. in `1.12.0` the
minor version is `1.12`.
The minor version is the first two numbers in the version string, e.g. in `1.13.0` the
minor version is `1.13`.

We try to track the latest version of Apache Flink at the time of the Beam release.
A Flink version is supported by Beam for the time it is supported by the Flink community.
Expand All @@ -326,7 +327,24 @@ To find out which version of Flink is compatible with Beam please see the table
<th>Artifact Id</th>
</tr>
<tr>
<td rowspan="3">&ge; 2.30.0</td>
<td rowspan="4">&ge; 2.31.0</td>
<td>1.13.x <sup>*</sup></td>
<td>beam-runners-flink-1.13</td>
</tr>
<tr>
<td>1.12.x <sup>*</sup></td>
<td>beam-runners-flink-1.12</td>
</tr>
<tr>
<td>1.11.x <sup>*</sup></td>
<td>beam-runners-flink-1.11</td>
</tr>
<tr>
<td>1.10.x</td>
<td>beam-runners-flink-1.10</td>
</tr>
<tr>
<td rowspan="3">2.30.0</td>
<td>1.12.x <sup>*</sup></td>
<td>beam-runners-flink-1.12</td>
</tr>
Expand Down

0 comments on commit 00eb420

Please sign in to comment.