zentol closed pull request #6416: [FLINK-9942][rest] Guard handlers against
null fields
URL: https://github.com/apache/flink/pull/6416
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/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java
index 0854ed724f1..491ba094f4c 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java
@@ -96,6 +96,13 @@ public JobSubmitHandler(
final JobSubmitRequestBody requestBody =
request.getRequestBody();
+ if (requestBody.jobGraphFileName == null) {
+ throw new RestHandlerException(
+ String.format("The %s field must not be omitted
or be null.",
+
JobSubmitRequestBody.FIELD_NAME_JOB_GRAPH),
+ HttpResponseStatus.BAD_REQUEST);
+ }
+
CompletableFuture<JobGraph> jobGraphFuture =
loadJobGraph(requestBody, nameToFile);
Collection<Path> jarFiles =
getJarFilesToUpload(requestBody.jarFileNames, nameToFile);
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointDisposalHandlers.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointDisposalHandlers.java
index 3cf5f59ca69..5c8bee2f8b4 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointDisposalHandlers.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointDisposalHandlers.java
@@ -21,6 +21,7 @@
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
import
org.apache.flink.runtime.rest.handler.async.AbstractAsynchronousOperationHandlers;
import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationInfo;
import org.apache.flink.runtime.rest.handler.async.OperationKey;
@@ -37,6 +38,8 @@
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.util.SerializedThrowable;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
import java.util.Map;
import java.util.concurrent.CompletableFuture;
@@ -64,8 +67,14 @@ public SavepointDisposalTriggerHandler(
}
@Override
- protected CompletableFuture<Acknowledge>
triggerOperation(HandlerRequest<SavepointDisposalRequest,
EmptyMessageParameters> request, RestfulGateway gateway) {
+ protected CompletableFuture<Acknowledge>
triggerOperation(HandlerRequest<SavepointDisposalRequest,
EmptyMessageParameters> request, RestfulGateway gateway) throws
RestHandlerException {
final String savepointPath =
request.getRequestBody().getSavepointPath();
+ if (savepointPath == null) {
+ throw new RestHandlerException(
+ String.format("Field %s must not be
omitted or be null.",
+
SavepointDisposalRequest.FIELD_NAME_SAVEPOINT_PATH),
+ HttpResponseStatus.BAD_REQUEST);
+ }
return gateway.disposeSavepoint(savepointPath,
RpcUtils.INF_TIMEOUT);
}
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitRequestBody.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitRequestBody.java
index 0ca82b4c78c..2026803a0ec 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitRequestBody.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitRequestBody.java
@@ -24,6 +24,7 @@
import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collection;
+import java.util.Collections;
import java.util.Objects;
/**
@@ -34,7 +35,7 @@
*/
public final class JobSubmitRequestBody implements RequestBody {
- private static final String FIELD_NAME_JOB_GRAPH = "jobGraphFileName";
+ public static final String FIELD_NAME_JOB_GRAPH = "jobGraphFileName";
private static final String FIELD_NAME_JOB_JARS = "jobJarFileNames";
private static final String FIELD_NAME_JOB_ARTIFACTS =
"jobArtifactFileNames";
@@ -53,8 +54,16 @@ public JobSubmitRequestBody(
@JsonProperty(FIELD_NAME_JOB_JARS) Collection<String>
jarFileNames,
@JsonProperty(FIELD_NAME_JOB_ARTIFACTS)
Collection<DistributedCacheFile> artifactFileNames) {
this.jobGraphFileName = jobGraphFileName;
- this.jarFileNames = jarFileNames;
- this.artifactFileNames = artifactFileNames;
+ if (jarFileNames == null) {
+ this.jarFileNames = Collections.emptyList();
+ } else {
+ this.jarFileNames = jarFileNames;
+ }
+ if (artifactFileNames == null) {
+ this.artifactFileNames = Collections.emptyList();
+ } else {
+ this.artifactFileNames = artifactFileNames;
+ }
}
@Override
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequest.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequest.java
index 229ae9107ab..9ceba36b6ef 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequest.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequest.java
@@ -31,7 +31,7 @@
*/
public class SavepointDisposalRequest implements RequestBody {
- private static final String FIELD_NAME_SAVEPOINT_PATH =
"savepoint-path";
+ public static final String FIELD_NAME_SAVEPOINT_PATH = "savepoint-path";
@JsonProperty(FIELD_NAME_SAVEPOINT_PATH)
private final String savepointPath;
----------------------------------------------------------------
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]
With regards,
Apache Git Services