Repository: flink Updated Branches: refs/heads/master 551653340 -> 541fe4366
[FLINK-8000] Sort Rest handler URLS in RestServerEndpoint Introduce special RestHandlerUrlComparator to sort REST URLs such that URLs with path parameters are sorted after those without or fewer. E.g. the following order would be established /jobs /jobs/overview /jobs/:jobid /jobs/:jobid/config /:* This closes #4958. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/34fdf569 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/34fdf569 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/34fdf569 Branch: refs/heads/master Commit: 34fdf569d7defb4393849fb9ecb2763b14532cc6 Parents: 5516533 Author: Till Rohrmann <trohrm...@apache.org> Authored: Sat Nov 4 14:56:11 2017 +0100 Committer: Till Rohrmann <trohrm...@apache.org> Committed: Thu Nov 9 10:48:38 2017 +0100 ---------------------------------------------------------------------- .../program/rest/RestClusterClientTest.java | 12 +-- .../dispatcher/DispatcherRestEndpoint.java | 6 +- .../flink/runtime/rest/RestServerEndpoint.java | 90 +++++++++++++++++++- .../flink/runtime/rest/RestEndpointITCase.java | 5 +- .../runtime/rest/RestServerEndpointTest.java | 63 ++++++++++++++ 5 files changed, 162 insertions(+), 14 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java ---------------------------------------------------------------------- diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index 0542d50..1bc04eb 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -111,9 +111,9 @@ public class RestClusterClientTest extends TestLogger { RestServerEndpoint rse = new RestServerEndpoint(rsec) { @Override - protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { + protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { - Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(); + List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(); handlers.add(Tuple2.of(portHandler.getMessageHeaders(), portHandler)); handlers.add(Tuple2.of(submitHandler.getMessageHeaders(), submitHandler)); handlers.add(Tuple2.of(terminationHandler.getMessageHeaders(), terminationHandler)); @@ -212,9 +212,9 @@ public class RestClusterClientTest extends TestLogger { RestServerEndpoint rse = new RestServerEndpoint(rsec) { @Override - protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { + protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { - Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(); + List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(); handlers.add(Tuple2.of(triggerHandler.getMessageHeaders(), triggerHandler)); return handlers; } @@ -280,9 +280,9 @@ public class RestClusterClientTest extends TestLogger { RestServerEndpoint rse = new RestServerEndpoint(rsec) { @Override - protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { + protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { - Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(); + List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(); handlers.add(Tuple2.of(listJobsHandler.getMessageHeaders(), listJobsHandler)); return handlers; } http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java index 6766784..a479749 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java @@ -85,7 +85,7 @@ import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -137,7 +137,7 @@ public class DispatcherRestEndpoint extends RestServerEndpoint { } @Override - protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { + protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { ArrayList<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(3); final Time timeout = restConfiguration.getTimeout(); @@ -352,8 +352,8 @@ public class DispatcherRestEndpoint extends RestServerEndpoint { handlers.add(Tuple2.of(jobSubmitHandler.getMessageHeaders(), jobSubmitHandler)); handlers.add(Tuple2.of(TaskManagersHeaders.getInstance(), taskManagersHandler)); handlers.add(Tuple2.of(TaskManagerDetailsHeaders.getInstance(), taskManagerDetailsHandler)); + handlers.add(Tuple2.of(SubtasksTimesHeaders.getInstance(), subtasksTimesHandler)); - // This handler MUST be added last, as it otherwise masks all subsequent GET handlers optWebContent.ifPresent( webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent))); http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index 18766c0..ea18e15 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -45,8 +45,11 @@ import org.slf4j.LoggerFactory; import javax.net.ssl.SSLEngine; +import java.io.Serializable; import java.net.InetSocketAddress; -import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -88,7 +91,7 @@ public abstract class RestServerEndpoint { * @param restAddressFuture future rest address of the RestServerEndpoint * @return Collection of AbstractRestHandler which are added to the server endpoint */ - protected abstract Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture); + protected abstract List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture); /** * Starts this REST server endpoint. @@ -107,7 +110,23 @@ public abstract class RestServerEndpoint { final Router router = new Router(); final CompletableFuture<String> restAddressFuture = new CompletableFuture<>(); - initializeHandlers(restAddressFuture).forEach(handler -> registerHandler(router, handler)); + List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = initializeHandlers(restAddressFuture); + + /* sort the handlers such that they are ordered the following: + * /jobs + * /jobs/overview + * /jobs/:jobid + * /jobs/:jobid/config + * /:* + */ + Collections.sort( + handlers, + RestHandlerUrlComparator.INSTANCE); + + handlers.forEach(handler -> { + log.debug("Register handler {} under {}@{}.", handler.f1, handler.f0.getHttpMethod(), handler.f0.getTargetRestEndpointURL()); + registerHandler(router, handler); + }); ChannelInitializer<SocketChannel> initializer = new ChannelInitializer<SocketChannel>() { @@ -268,4 +287,69 @@ public abstract class RestServerEndpoint { throw new RuntimeException("Unsupported http method: " + specificationHandler.f0.getHttpMethod() + '.'); } } + + /** + * Comparator for Rest URLs. + * + * <p>The comparator orders the Rest URLs such that URLs with path parameters are ordered behind + * those without parameters. E.g.: + * /jobs + * /jobs/overview + * /jobs/:jobid + * /jobs/:jobid/config + * /:* + * + * <p>IMPORTANT: This comparator is highly specific to how Netty path parameter are encoded. Namely + * via a preceding ':' character. + */ + static final class RestHandlerUrlComparator implements Comparator<Tuple2<RestHandlerSpecification, ChannelInboundHandler>>, Serializable { + + private static final long serialVersionUID = 2388466767835547926L; + + private static final Comparator<String> CASE_INSENSITIVE_ORDER = new CaseInsensitiveOrderComparator(); + + static final RestHandlerUrlComparator INSTANCE = new RestHandlerUrlComparator(); + + @Override + public int compare( + Tuple2<RestHandlerSpecification, ChannelInboundHandler> o1, + Tuple2<RestHandlerSpecification, ChannelInboundHandler> o2) { + return CASE_INSENSITIVE_ORDER.compare(o1.f0.getTargetRestEndpointURL(), o2.f0.getTargetRestEndpointURL()); + } + + static final class CaseInsensitiveOrderComparator implements Comparator<String>, Serializable { + private static final long serialVersionUID = 8550835445193437027L; + + @Override + public int compare(String s1, String s2) { + int n1 = s1.length(); + int n2 = s2.length(); + int min = Math.min(n1, n2); + for (int i = 0; i < min; i++) { + char c1 = s1.charAt(i); + char c2 = s2.charAt(i); + if (c1 != c2) { + c1 = Character.toUpperCase(c1); + c2 = Character.toUpperCase(c2); + if (c1 != c2) { + c1 = Character.toLowerCase(c1); + c2 = Character.toLowerCase(c2); + if (c1 != c2) { + if (c1 == ':') { + // c2 is less than c1 because it is also different + return 1; + } else if (c2 == ':') { + // c1 is less than c2 + return -1; + } else { + return c1 - c2; + } + } + } + } + } + return n1 - n2; + } + } + } } http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/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 4d3c6b5..ee59da7 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 @@ -57,6 +57,7 @@ import javax.annotation.Nonnull; import java.net.InetSocketAddress; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -205,8 +206,8 @@ public class RestEndpointITCase extends TestLogger { } @Override - protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { - return Collections.singleton(Tuple2.of(new TestHeaders(), testHandler)); + protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { + return Collections.singletonList(Tuple2.of(new TestHeaders(), testHandler)); } } http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java ---------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java new file mode 100644 index 0000000..3cb4a55 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest; + +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +/** + * Test cases for the {@link RestServerEndpoint}. + */ +public class RestServerEndpointTest extends TestLogger { + + /** + * Tests that the REST handler URLs are properly sorted. + */ + @Test + public void testRestHandlerUrlSorting() { + final int numberHandlers = 5; + + final List<String> handlerUrls = new ArrayList<>(numberHandlers); + + handlerUrls.add("/jobs/overview"); + handlerUrls.add("/jobs/:jobid"); + handlerUrls.add("/jobs"); + handlerUrls.add("/:*"); + handlerUrls.add("/jobs/:jobid/config"); + + final List<String> expected = new ArrayList<>(numberHandlers); + + expected.add("/jobs"); + expected.add("/jobs/overview"); + expected.add("/jobs/:jobid"); + expected.add("/jobs/:jobid/config"); + expected.add("/:*"); + + Collections.sort(handlerUrls, new RestServerEndpoint.RestHandlerUrlComparator.CaseInsensitiveOrderComparator()); + + assertEquals(expected, handlerUrls); + } +}