This is an automated email from the ASF dual-hosted git repository.

guoweijie pushed a commit to branch release-1.17
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.17 by this push:
     new c0037099165 [FLINK-31812][api] Transform savepoint path to null if 
empty in body
c0037099165 is described below

commit c003709916540aa272e15a4c8df4374f594ebad7
Author: Nicolas Fraison <[email protected]>
AuthorDate: Tue Apr 18 11:26:36 2023 +0200

    [FLINK-31812][api] Transform savepoint path to null if empty in body
---
 .../runtime/webmonitor/handlers/JarRunHandler.java |  2 +-
 .../handlers/JarRunHandlerParameterTest.java       | 29 ++++++++++++++++++++++
 2 files changed, 30 insertions(+), 1 deletion(-)

diff --git 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
index 8a04a1279f9..2b30b699b09 100644
--- 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
+++ 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
@@ -142,7 +142,7 @@ public class JarRunHandler
                         log);
         final String savepointPath =
                 fromRequestBodyOrQueryParameter(
-                        requestBody.getSavepointPath(),
+                        emptyToNull(requestBody.getSavepointPath()),
                         () ->
                                 emptyToNull(
                                         getQueryParameter(
diff --git 
a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java
 
b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java
index d46eb222486..2d236335163 100644
--- 
a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java
+++ 
b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java
@@ -202,6 +202,20 @@ class JarRunHandlerParameterTest
                 FLINK_CONFIGURATION.toMap());
     }
 
+    private JarRunRequestBody getJarRequestBodyWithSavepointPath(
+            ProgramArgsParType programArgsParType, String savepointPath) {
+        return new JarRunRequestBody(
+                ParameterProgram.class.getCanonicalName(),
+                getProgramArgsString(programArgsParType),
+                getProgramArgsList(programArgsParType),
+                PARALLELISM,
+                null,
+                ALLOW_NON_RESTORED_STATE_QUERY,
+                savepointPath,
+                RESTORE_MODE,
+                null);
+    }
+
     @Override
     JarRunRequestBody getJarRequestBodyWithJobId(JobID jobId) {
         return new JarRunRequestBody(null, null, null, null, jobId, null, 
null, null, null);
@@ -248,6 +262,21 @@ class JarRunHandlerParameterTest
                         });
     }
 
+    @Test
+    void testConfigurationWithEmptySavepointPath() throws Exception {
+        final JarRunRequestBody requestBody =
+                getJarRequestBodyWithSavepointPath(ProgramArgsParType.String, 
"");
+        handleRequest(
+                createRequest(
+                        requestBody,
+                        getUnresolvedJarMessageParameters(),
+                        getUnresolvedJarMessageParameters(),
+                        jarWithEagerSink));
+        JobGraph jobGraph = LAST_SUBMITTED_JOB_GRAPH_REFERENCE.get();
+        assertThat(jobGraph.getSavepointRestoreSettings())
+                .isEqualTo(SavepointRestoreSettings.none());
+    }
+
     @Override
     void handleRequest(HandlerRequest<JarRunRequestBody> request) throws 
Exception {
         handler.handleRequest(request, restfulGateway).get();

Reply via email to