You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2018/07/27 10:36:46 UTC

[flink] branch release-1.5 updated: [FLINK-9942][rest] Guard handlers against null fields

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

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


The following commit(s) were added to refs/heads/release-1.5 by this push:
     new ee483b0  [FLINK-9942][rest] Guard handlers against null fields
ee483b0 is described below

commit ee483b0a0d1807fc06ceb5bbfd9fd2fa8b2d1480
Author: zentol <ch...@apache.org>
AuthorDate: Wed Jul 25 08:39:29 2018 +0200

    [FLINK-9942][rest] Guard handlers against null fields
---
 .../runtime/rest/handler/job/JobSubmitHandler.java     |  7 +++++++
 .../job/savepoints/SavepointDisposalHandlers.java      | 11 ++++++++++-
 .../rest/messages/job/JobSubmitRequestBody.java        | 18 ++++++++++++++----
 .../job/savepoints/SavepointDisposalRequest.java       |  7 ++++---
 4 files changed, 35 insertions(+), 8 deletions(-)

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 2821331..ccb34f9 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
@@ -95,6 +95,13 @@ public final class JobSubmitHandler extends AbstractRestHandler<DispatcherGatewa
 
 		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 3cf5f59..5c8bee2 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 @@ package org.apache.flink.runtime.rest.handler.job.savepoints;
 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.RestfulGateway;
 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 class SavepointDisposalHandlers extends AbstractAsynchronousOperationHand
 		}
 
 		@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 8829bc8..8f88b9b 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
@@ -23,7 +23,11 @@ import org.apache.flink.runtime.rest.messages.RequestBody;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Objects;
 
 /**
@@ -34,21 +38,27 @@ import java.util.Objects;
  */
 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";
 
 	@JsonProperty(FIELD_NAME_JOB_GRAPH)
+	@Nullable
 	public final String jobGraphFileName;
 
 	@JsonProperty(FIELD_NAME_JOB_JARS)
+	@Nonnull
 	public final Collection<String> jarFileNames;
 
 	@JsonCreator
 	public JobSubmitRequestBody(
-			@JsonProperty(FIELD_NAME_JOB_GRAPH) String jobGraphFileName,
-			@JsonProperty(FIELD_NAME_JOB_JARS) Collection<String> jarFileNames) {
+			@Nullable @JsonProperty(FIELD_NAME_JOB_GRAPH) String jobGraphFileName,
+			@Nullable @JsonProperty(FIELD_NAME_JOB_JARS) Collection<String> jarFileNames) {
 		this.jobGraphFileName = jobGraphFileName;
-		this.jarFileNames = jarFileNames;
+		if (jarFileNames == null) {
+			this.jarFileNames = Collections.emptyList();
+		} else {
+			this.jarFileNames = jarFileNames;
+		}
 	}
 
 	@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 229ae91..2026c19 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
@@ -24,21 +24,22 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCre
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
-import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 
 /**
  * Request body for a savepoint disposal call.
  */
 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)
+	@Nullable
 	private final String savepointPath;
 
 	@JsonCreator
 	public SavepointDisposalRequest(
-		@JsonProperty(FIELD_NAME_SAVEPOINT_PATH) @Nonnull String savepointPath) {
+			@Nullable @JsonProperty(FIELD_NAME_SAVEPOINT_PATH) String savepointPath) {
 		this.savepointPath = savepointPath;
 	}