Repository: flink Updated Branches: refs/heads/master 8706c6f44 -> 6b3fdc288
[FLINK-7790] [REST] Unresolved query params not added to request URL This close s#4788. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6b3fdc28 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6b3fdc28 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6b3fdc28 Branch: refs/heads/master Commit: 6b3fdc288587fe0982f2ffa2e476e0fd3cd61188 Parents: 8706c6f Author: zentol <[email protected]> Authored: Tue Oct 10 12:57:05 2017 +0200 Committer: zentol <[email protected]> Committed: Tue Oct 10 16:56:32 2017 +0200 ---------------------------------------------------------------------- .../runtime/rest/messages/MessageParameters.java | 2 +- .../rest/messages/MessageParametersTest.java | 19 ++++++++++++++++++- 2 files changed, 19 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/6b3fdc28/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageParameters.java ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageParameters.java index 96243c1..b19b12e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageParameters.java @@ -84,7 +84,7 @@ public abstract class MessageParameters { } boolean isFirstQueryParameter = true; for (MessageQueryParameter<?> queryParameter : parameters.getQueryParameters()) { - if (parameters.isResolved()) { + if (queryParameter.isResolved()) { if (isFirstQueryParameter) { queryParameters.append('?'); isFirstQueryParameter = false; http://git-wip-us.apache.org/repos/asf/flink/blob/6b3fdc28/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java ---------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java index 7458821..2da7455 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java @@ -45,6 +45,23 @@ public class MessageParametersTest extends TestLogger { Assert.assertEquals("/jobs/" + pathJobID + "/state?jobid=" + queryJobID, resolvedUrl); } + @Test + public void testUnresolvedParameters() { + String genericUrl = "/jobs/:jobid/state"; + TestMessageParameters parameters = new TestMessageParameters(); + try { + MessageParameters.resolveUrl(genericUrl, parameters); + Assert.fail(); + } catch (IllegalStateException expected) { + // the mandatory jobid path parameter was not resolved + } + JobID jobID = new JobID(); + parameters.pathParameter.resolve(jobID); + + String resolvedUrl = MessageParameters.resolveUrl(genericUrl, parameters); + Assert.assertEquals("/jobs/" + jobID + "/state", resolvedUrl); + } + private static class TestMessageParameters extends MessageParameters { private final TestPathParameter pathParameter = new TestPathParameter(); private final TestQueryParameter queryParameter = new TestQueryParameter(); @@ -80,7 +97,7 @@ public class MessageParametersTest extends TestLogger { private static class TestQueryParameter extends MessageQueryParameter<JobID> { TestQueryParameter() { - super("jobid", MessageParameterRequisiteness.MANDATORY); + super("jobid", MessageParameterRequisiteness.OPTIONAL); } @Override
