This is an automated email from the ASF dual-hosted git repository. chesnay pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit 464991810c01d5e4e42e41159ccfd6f1ded7623b Author: Chesnay Schepler <[email protected]> AuthorDate: Mon Oct 4 11:46:07 2021 +0200 [FLINK-24275][rest] Refactor parameter resolving into separate methods --- .../flink/runtime/rest/handler/HandlerRequest.java | 118 +++++++++++++-------- .../job/metrics/AbstractMetricsHandlerTest.java | 4 +- .../rest/handler/util/HandlerRequestUtilsTest.java | 4 +- 3 files changed, 77 insertions(+), 49 deletions(-) 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 8b92c1b..61977e6 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 @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rest.handler; +import org.apache.flink.runtime.rest.messages.MessageParameter; import org.apache.flink.runtime.rest.messages.MessageParameters; import org.apache.flink.runtime.rest.messages.MessagePathParameter; import org.apache.flink.runtime.rest.messages.MessageQueryParameter; @@ -46,9 +47,9 @@ public class HandlerRequest<R extends RequestBody, M extends MessageParameters> private final R requestBody; private final Collection<File> uploadedFiles; private final Map<Class<? extends MessagePathParameter<?>>, MessagePathParameter<?>> - pathParameters = new HashMap<>(2); + pathParameters; private final Map<Class<? extends MessageQueryParameter<?>>, MessageQueryParameter<?>> - queryParameters = new HashMap<>(2); + queryParameters; public HandlerRequest(R requestBody, M messageParameters) throws HandlerRequestException { this( @@ -87,50 +88,8 @@ public class HandlerRequest<R extends RequestBody, M extends MessageParameters> Preconditions.checkNotNull(receivedQueryParameters); Preconditions.checkNotNull(receivedPathParameters); - for (MessagePathParameter<?> pathParameter : messageParameters.getPathParameters()) { - String value = receivedPathParameters.get(pathParameter.getKey()); - if (value != null) { - try { - pathParameter.resolveFromString(value); - } catch (Exception e) { - throw new HandlerRequestException( - "Cannot resolve path parameter (" - + pathParameter.getKey() - + ") from value \"" - + value - + "\"."); - } - - @SuppressWarnings("unchecked") - Class<? extends MessagePathParameter<?>> clazz = - (Class<? extends MessagePathParameter<?>>) pathParameter.getClass(); - pathParameters.put(clazz, pathParameter); - } - } - - for (MessageQueryParameter<?> queryParameter : messageParameters.getQueryParameters()) { - List<String> values = receivedQueryParameters.get(queryParameter.getKey()); - if (values != null && !values.isEmpty()) { - StringJoiner joiner = new StringJoiner(","); - values.forEach(joiner::add); - - try { - queryParameter.resolveFromString(joiner.toString()); - } catch (Exception e) { - throw new HandlerRequestException( - "Cannot resolve query parameter (" - + queryParameter.getKey() - + ") from value \"" - + joiner - + "\"."); - } - - @SuppressWarnings("unchecked") - Class<? extends MessageQueryParameter<?>> clazz = - (Class<? extends MessageQueryParameter<?>>) queryParameter.getClass(); - queryParameters.put(clazz, queryParameter); - } - } + pathParameters = resolvePathParameters(messageParameters, receivedPathParameters); + queryParameters = resolveQueryParameters(messageParameters, receivedQueryParameters); } /** @@ -183,4 +142,71 @@ public class HandlerRequest<R extends RequestBody, M extends MessageParameters> public Collection<File> getUploadedFiles() { return uploadedFiles; } + + private static Map<Class<? extends MessagePathParameter<?>>, MessagePathParameter<?>> + resolvePathParameters( + MessageParameters messageParameters, Map<String, String> receivedPathParameters) + throws HandlerRequestException { + + for (MessagePathParameter<?> pathParameter : messageParameters.getPathParameters()) { + String value = receivedPathParameters.get(pathParameter.getKey()); + if (value != null) { + try { + pathParameter.resolveFromString(value); + } catch (Exception e) { + throw new HandlerRequestException( + "Cannot resolve path parameter (" + + pathParameter.getKey() + + ") from value \"" + + value + + "\"."); + } + } + } + + return mapParameters(messageParameters.getPathParameters()); + } + + private static Map<Class<? extends MessageQueryParameter<?>>, MessageQueryParameter<?>> + resolveQueryParameters( + MessageParameters messageParameters, + Map<String, List<String>> receivedQueryParameters) + throws HandlerRequestException { + + for (MessageQueryParameter<?> queryParameter : messageParameters.getQueryParameters()) { + List<String> values = receivedQueryParameters.get(queryParameter.getKey()); + if (values != null && !values.isEmpty()) { + StringJoiner joiner = new StringJoiner(","); + values.forEach(joiner::add); + + try { + queryParameter.resolveFromString(joiner.toString()); + } catch (Exception e) { + throw new HandlerRequestException( + "Cannot resolve query parameter (" + + queryParameter.getKey() + + ") from value \"" + + joiner + + "\"."); + } + } + } + + return mapParameters(messageParameters.getQueryParameters()); + } + + private static <P extends MessageParameter<?>> Map<Class<? extends P>, P> mapParameters( + Collection<P> pathParameters) { + final Map<Class<? extends P>, P> mappedParameters = new HashMap<>(2); + + for (P pathParameter : pathParameters) { + if (pathParameter.isResolved()) { + @SuppressWarnings("unchecked") + Class<P> clazz = (Class<P>) pathParameter.getClass(); + mappedParameters.put(clazz, pathParameter); + } + } + + return mappedParameters; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java index 6447973..04b5f5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java @@ -227,6 +227,8 @@ public class AbstractMetricsHandlerTest extends TestLogger { private static class TestMessageParameters extends MessageParameters { + private final MetricsFilterParameter metricsFilterParameter = new MetricsFilterParameter(); + @Override public Collection<MessagePathParameter<?>> getPathParameters() { return Collections.emptyList(); @@ -234,7 +236,7 @@ public class AbstractMetricsHandlerTest extends TestLogger { @Override public Collection<MessageQueryParameter<?>> getQueryParameters() { - return Collections.singletonList(new MetricsFilterParameter()); + return Collections.singletonList(metricsFilterParameter); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java index 9279932..752a406 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java @@ -83,7 +83,8 @@ public class HandlerRequestUtilsTest extends TestLogger { private static class TestMessageParameters extends MessageParameters { - private TestBooleanQueryParameter testBooleanQueryParameter; + private final TestBooleanQueryParameter testBooleanQueryParameter = + new TestBooleanQueryParameter(); @Override public Collection<MessagePathParameter<?>> getPathParameters() { @@ -92,7 +93,6 @@ public class HandlerRequestUtilsTest extends TestLogger { @Override public Collection<MessageQueryParameter<?>> getQueryParameters() { - testBooleanQueryParameter = new TestBooleanQueryParameter(); return Collections.singletonList(testBooleanQueryParameter); } }
