[
https://issues.apache.org/jira/browse/BEAM-5396?focusedWorklogId=171516&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-171516
]
ASF GitHub Bot logged work on BEAM-5396:
----------------------------------------
Author: ASF GitHub Bot
Created on: 03/Dec/18 10:08
Start Date: 03/Dec/18 10:08
Worklog Time Spent: 10m
Work Description: mxm closed pull request #7169: [BEAM-5396] Savepoint
restore option in Flink runner
URL: https://github.com/apache/beam/pull/7169
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
index 9802defeb1a4..80a9ef860a4c 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
@@ -19,19 +19,31 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Splitter;
+import java.net.URL;
+import java.util.Collections;
import java.util.List;
import javax.annotation.Nullable;
import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.java.CollectionEnvironment;
import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.JobWithJars;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.client.program.StandaloneClusterClient;
+import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
import org.apache.flink.runtime.state.StateBackend;
import org.apache.flink.streaming.api.TimeCharacteristic;
import
org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup;
+import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.graph.StreamGraph;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -128,7 +140,7 @@ static StreamExecutionEnvironment
createStreamExecutionEnvironment(
String masterUrl = options.getFlinkMaster();
Configuration flinkConfig = getFlinkConfiguration(confDir);
- StreamExecutionEnvironment flinkStreamEnv = null;
+ final StreamExecutionEnvironment flinkStreamEnv;
// depending on the master, create the right environment.
if ("[local]".equals(masterUrl)) {
@@ -138,11 +150,22 @@ static StreamExecutionEnvironment
createStreamExecutionEnvironment(
} else if (masterUrl.matches(".*:\\d*")) {
List<String> parts = Splitter.on(':').splitToList(masterUrl);
flinkConfig.setInteger(RestOptions.PORT, Integer.parseInt(parts.get(1)));
+
+ final SavepointRestoreSettings savepointRestoreSettings;
+ if (options.getSavepointPath() != null) {
+ savepointRestoreSettings =
+ SavepointRestoreSettings.forPath(
+ options.getSavepointPath(),
options.getAllowNonRestoredState());
+ } else {
+ savepointRestoreSettings = SavepointRestoreSettings.none();
+ }
+
flinkStreamEnv =
- StreamExecutionEnvironment.createRemoteEnvironment(
+ new BeamFlinkRemoteStreamEnvironment(
parts.get(0),
Integer.parseInt(parts.get(1)),
flinkConfig,
+ savepointRestoreSettings,
filesToStage.toArray(new String[filesToStage.size()]));
} else {
LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].",
masterUrl);
@@ -263,4 +286,82 @@ private static void applyLatencyTrackingInterval(
long latencyTrackingInterval = options.getLatencyTrackingInterval();
config.setLatencyTrackingInterval(latencyTrackingInterval);
}
+
+ /**
+ * Remote stream environment that supports job execution with restore from
savepoint.
+ *
+ * <p>This class can be removed once Flink provides this functionality.
+ *
+ * <p>TODO: https://issues.apache.org/jira/browse/BEAM-5396
+ */
+ private static class BeamFlinkRemoteStreamEnvironment extends
RemoteStreamEnvironment {
+ private final SavepointRestoreSettings restoreSettings;
+
+ public BeamFlinkRemoteStreamEnvironment(
+ String host,
+ int port,
+ Configuration clientConfiguration,
+ SavepointRestoreSettings restoreSettings,
+ String... jarFiles) {
+ super(host, port, clientConfiguration, jarFiles, null);
+ this.restoreSettings = restoreSettings;
+ }
+
+ // copied from RemoteStreamEnvironment and augmented to pass savepoint
restore settings
+ @Override
+ protected JobExecutionResult executeRemotely(StreamGraph streamGraph,
List<URL> jarFiles)
+ throws ProgramInvocationException {
+
+ List<URL> globalClasspaths = Collections.emptyList();
+ String host = super.getHost();
+ int port = super.getPort();
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Running remotely at {}:{}", host, port);
+ }
+
+ ClassLoader usercodeClassLoader =
+ JobWithJars.buildUserCodeClassLoader(
+ jarFiles, globalClasspaths, getClass().getClassLoader());
+
+ Configuration configuration = new Configuration();
+ configuration.addAll(super.getClientConfiguration());
+
+ configuration.setString(JobManagerOptions.ADDRESS, host);
+ configuration.setInteger(JobManagerOptions.PORT, port);
+
+ configuration.setInteger(RestOptions.PORT, port);
+
+ final ClusterClient<?> client;
+ try {
+ if
(CoreOptions.LEGACY_MODE.equals(configuration.getString(CoreOptions.MODE))) {
+ client = new StandaloneClusterClient(configuration);
+ } else {
+ client = new RestClusterClient<>(configuration,
"RemoteStreamEnvironment");
+ }
+ } catch (Exception e) {
+ throw new ProgramInvocationException(
+ "Cannot establish connection to JobManager: " + e.getMessage(), e);
+ }
+
+
client.setPrintStatusDuringExecution(getConfig().isSysoutLoggingEnabled());
+
+ try {
+ return client
+ .run(streamGraph, jarFiles, globalClasspaths, usercodeClassLoader,
restoreSettings)
+ .getJobExecutionResult();
+ } catch (ProgramInvocationException e) {
+ throw e;
+ } catch (Exception e) {
+ String term = e.getMessage() == null ? "." : (": " + e.getMessage());
+ throw new ProgramInvocationException("The program execution failed" +
term, e);
+ } finally {
+ try {
+ client.shutdown();
+ } catch (Exception e) {
+ LOG.warn("Could not properly shut down the cluster client.", e);
+ }
+ }
+ }
+ }
}
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
index 5c55e78218f4..6c7c5b5b6b4b 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -204,4 +204,18 @@
ExecutionMode getExecutionModeForBatch();
void setExecutionModeForBatch(ExecutionMode executionMode);
+
+ @Description(
+ "Savepoint restore path. If specified, restores the streaming pipeline
from the provided path.")
+ String getSavepointPath();
+
+ void setSavepointPath(String path);
+
+ @Description(
+ "Flag indicating whether non restored state is allowed if the savepoint "
+ + "contains state for an operator that is no longer part of the
pipeline.")
+ @Default.Boolean(false)
+ Boolean getAllowNonRestoredState();
+
+ void setAllowNonRestoredState(Boolean allowNonRestoredState);
}
diff --git
a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
index e3198dc1bfb6..f8131e4de934 100644
---
a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
+++
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
@@ -18,6 +18,7 @@
package org.apache.beam.runners.flink;
import static org.hamcrest.core.Is.is;
+import static org.hamcrest.core.IsInstanceOf.instanceOf;
import static org.junit.Assert.assertThat;
import java.io.File;
@@ -27,11 +28,14 @@
import java.util.Collections;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
+import org.mockito.internal.util.reflection.Whitebox;
/** Tests for {@link FlinkExecutionEnvironments}. */
public class FlinkExecutionEnvironmentsTest {
@@ -158,4 +162,22 @@ private String extractFlinkConfig() throws IOException {
Files.copy(inputStream, new File(root, "flink-conf.yaml").toPath());
return root.getAbsolutePath();
}
+
+ @Test
+ public void shouldSetSavepointRestoreForRemoteStreaming() {
+ String path = "fakePath";
+ FlinkPipelineOptions options =
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+ options.setRunner(TestFlinkRunner.class);
+ options.setFlinkMaster("host:80");
+ options.setSavepointPath(path);
+
+ StreamExecutionEnvironment sev =
+ FlinkExecutionEnvironments.createStreamExecutionEnvironment(
+ options, Collections.emptyList());
+ // subject to change with https://issues.apache.org/jira/browse/FLINK-11048
+ assertThat(sev, instanceOf(RemoteStreamEnvironment.class));
+ assertThat(
+ Whitebox.getInternalState(sev, "restoreSettings"),
+ is(SavepointRestoreSettings.forPath(path)));
+ }
}
diff --git
a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
index 257501fd6586..a65e39a1dcf3 100644
---
a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
+++
b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
@@ -86,6 +86,8 @@ public void testDefaults() {
assertThat(options.getMaxBundleSize(), is(1000L));
assertThat(options.getMaxBundleTimeMills(), is(1000L));
assertThat(options.getExecutionModeForBatch(),
is(ExecutionMode.PIPELINED));
+ assertThat(options.getSavepointPath(), is(nullValue()));
+ assertThat(options.getAllowNonRestoredState(), is(false));
}
@Test(expected = Exception.class)
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 171516)
Time Spent: 2h 10m (was: 2h)
> Flink portable runner savepoint / upgrade support
> -------------------------------------------------
>
> Key: BEAM-5396
> URL: https://issues.apache.org/jira/browse/BEAM-5396
> Project: Beam
> Issue Type: Improvement
> Components: runner-flink
> Reporter: Thomas Weise
> Assignee: Thomas Weise
> Priority: Major
> Labels: portability, portability-flink
> Time Spent: 2h 10m
> Remaining Estimate: 0h
>
> The portable Flink runner needs to support Flink savepoints for production
> use. It should be possible to upgrade a stateful portable Beam pipeline that
> runs on Flink, which involves taking a savepoint and then starting the new
> version of the pipeline from that savepoint. The potential issues with
> pipeline evolution and migration are similar to those when using the Flink
> DataStream API (schema / name changes etc.).
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)