[FLINK-7543] [REST] Simplify handler access to path/query parameters This closes #4611.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/a6905df0 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/a6905df0 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/a6905df0 Branch: refs/heads/master Commit: a6905df09098caf6c2b3c11e164132784801d815 Parents: dcce0b7 Author: zentol <ches...@apache.org> Authored: Mon Aug 28 17:26:01 2017 +0200 Committer: zentol <ches...@apache.org> Committed: Tue Aug 29 12:07:05 2017 +0200 ---------------------------------------------------------------------- .../runtime/rest/handler/HandlerRequest.java | 29 +++++++++++++------- .../flink/runtime/rest/RestEndpointITCase.java | 12 ++------ 2 files changed, 21 insertions(+), 20 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/a6905df0/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java index fa17b24..6a9bce9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.rest.messages.MessageQueryParameter; import org.apache.flink.runtime.rest.messages.RequestBody; import org.apache.flink.util.Preconditions; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -83,28 +84,36 @@ public class HandlerRequest<R extends RequestBody, M extends MessageParameters> } /** - * Returns the {@link MessagePathParameter} for the given class. + * Returns the value of the {@link MessagePathParameter} for the given class. * * @param parameterClass class of the parameter * @param <X> the value type that the parameter contains * @param <PP> type of the path parameter - * @return path parameter for the given class, or null if no parameter value exists for the given class + * @return path parameter value for the given class + * @throws IllegalStateException if no value is defined for the given parameter class */ - @SuppressWarnings("unchecked") - public <X, PP extends MessagePathParameter<X>> PP getPathParameter(Class<PP> parameterClass) { - return (PP) pathParameters.get(parameterClass); + public <X, PP extends MessagePathParameter<X>> X getPathParameter(Class<PP> parameterClass) { + @SuppressWarnings("unchecked") + PP pathParameter = (PP) pathParameters.get(parameterClass); + Preconditions.checkState(pathParameter != null, "No parameter could be found for the given class."); + return pathParameter.getValue(); } /** - * Returns the {@link MessageQueryParameter} for the given class. + * Returns the value of the {@link MessageQueryParameter} for the given class. * * @param parameterClass class of the parameter * @param <X> the value type that the parameter contains * @param <QP> type of the query parameter - * @return query parameter for the given class, or null if no parameter value exists for the given class + * @return query parameter value for the given class, or an empty list if no parameter value exists for the given class */ - @SuppressWarnings("unchecked") - public <X, QP extends MessageQueryParameter<X>> QP getQueryParameter(Class<QP> parameterClass) { - return (QP) queryParameters.get(parameterClass); + public <X, QP extends MessageQueryParameter<X>> List<X> getQueryParameter(Class<QP> parameterClass) { + @SuppressWarnings("unchecked") + QP queryParameter = (QP) queryParameters.get(parameterClass); + if (queryParameter == null) { + return Collections.emptyList(); + } else { + return queryParameter.getValue(); + } } } http://git-wip-us.apache.org/repos/asf/flink/blob/a6905df0/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java ---------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java index 77c5179..ab43f77 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java @@ -131,16 +131,8 @@ public class RestEndpointITCase extends TestLogger { @Override protected CompletableFuture<TestResponse> handleRequest(@Nonnull HandlerRequest<TestRequest, TestParameters> request) throws RestHandlerException { - if (request.getPathParameter(JobIDPathParameter.class) == null) { - throw new RestHandlerException("Path parameter was missing.", HttpResponseStatus.INTERNAL_SERVER_ERROR); - } else { - Assert.assertEquals(request.getPathParameter(JobIDPathParameter.class).getValue(), PATH_JOB_ID); - } - if (request.getQueryParameter(JobIDQueryParameter.class) == null) { - throw new RestHandlerException("Query parameter was missing.", HttpResponseStatus.INTERNAL_SERVER_ERROR); - } else { - Assert.assertEquals(request.getQueryParameter(JobIDQueryParameter.class).getValue().get(0), QUERY_JOB_ID); - } + Assert.assertEquals(request.getPathParameter(JobIDPathParameter.class), PATH_JOB_ID); + Assert.assertEquals(request.getQueryParameter(JobIDQueryParameter.class).get(0), QUERY_JOB_ID); if (request.getRequestBody().id == 1) { synchronized (LOCK) {