Skip to content

Commit 92324ed

Browse files
committed
[Fix] 修复JAR任务从Savepoint恢复时allowNonRestoredState未生效的问题
1 parent 5e98cf1 commit 92324ed

File tree

1 file changed

+20
-11
lines changed

1 file changed

+20
-11
lines changed

dinky-core/src/main/java/org/dinky/job/runner/JobJarRunner.java

Lines changed: 20 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,6 @@
5353
import org.apache.flink.core.execution.JobClient;
5454
import org.apache.flink.runtime.jobgraph.JobGraph;
5555
import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
56-
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
5756
import org.apache.flink.streaming.api.graph.StreamGraph;
5857

5958
import java.io.File;
@@ -62,6 +61,7 @@
6261
import java.util.ArrayList;
6362
import java.util.Collections;
6463
import java.util.List;
64+
import java.util.Map;
6565
import java.util.Set;
6666

6767
import cn.hutool.core.lang.Assert;
@@ -170,16 +170,25 @@ private GatewayResult submitNormalWithGateway(JobStatement jobStatement) {
170170
private Pipeline getPipeline(JobStatement jobStatement) {
171171
Pipeline pipeline = getJarStreamGraph(jobStatement.getStatement(), jobManager.getDinkyClassLoader());
172172
if (pipeline instanceof StreamGraph) {
173-
if (Asserts.isNotNullString(jobManager.getConfig().getSavePointPath())
174-
|| (Asserts.isNotNull(jobManager.getConfig().getConfigJson())
175-
&& Asserts.isNotNullString(jobManager
176-
.getConfig()
177-
.getConfigJson()
178-
.get(SavepointConfigOptions.SAVEPOINT_PATH)))) {
179-
((StreamGraph) pipeline)
180-
.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(
181-
jobManager.getConfig().getSavePointPath(),
182-
configuration.get(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE)));
173+
String savePointPath = jobManager.getConfig().getSavePointPath();
174+
Map<String, String> configJson = jobManager.getConfig().getConfigJson();
175+
if (Asserts.isNotNullString(savePointPath)
176+
|| (Asserts.isNotNull(configJson)
177+
&& Asserts.isNotNullString(configJson.get(SavepointConfigOptions.SAVEPOINT_PATH.key())))) {
178+
String effectivePath = Asserts.isNotNullString(savePointPath)
179+
? savePointPath
180+
: configJson.get(SavepointConfigOptions.SAVEPOINT_PATH.key());
181+
JarSubmitParam submitParam = JarSubmitParam.getInfo(jobStatement.getStatement());
182+
boolean allowNonRestored = Boolean.TRUE.equals(submitParam.getAllowNonRestoredState());
183+
log.info(
184+
"Setting savepoint restore settings, path: {}, allowNonRestoredState: {}",
185+
effectivePath,
186+
allowNonRestored);
187+
// 设置到 getRootConfiguration(),executeAsync 会以它为基础创建拷贝
188+
Configuration rootConfiguration =
189+
executor.getCustomTableEnvironment().getRootConfiguration();
190+
rootConfiguration.set(SavepointConfigOptions.SAVEPOINT_PATH, effectivePath);
191+
rootConfiguration.set(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE, allowNonRestored);
183192
}
184193
}
185194
return pipeline;

0 commit comments

Comments
 (0)