Skip to content

Commit

Permalink
[FLINK-8000] Sort Rest handler URLS in RestServerEndpoint
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
tillrohrmann committed Nov 9, 2017
1 parent 5516533 commit 34fdf56
Show file tree
Hide file tree
Showing 5 changed files with 162 additions and 14 deletions.
Original file line number Original file line Diff line number Diff line change
Expand Up @@ -111,9 +111,9 @@ public void testJobSubmitCancelStop() throws Exception {


RestServerEndpoint rse = new RestServerEndpoint(rsec) { RestServerEndpoint rse = new RestServerEndpoint(rsec) {
@Override @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(portHandler.getMessageHeaders(), portHandler));
handlers.add(Tuple2.of(submitHandler.getMessageHeaders(), submitHandler)); handlers.add(Tuple2.of(submitHandler.getMessageHeaders(), submitHandler));
handlers.add(Tuple2.of(terminationHandler.getMessageHeaders(), terminationHandler)); handlers.add(Tuple2.of(terminationHandler.getMessageHeaders(), terminationHandler));
Expand Down Expand Up @@ -212,9 +212,9 @@ public void testTriggerSavepoint() throws Exception {


RestServerEndpoint rse = new RestServerEndpoint(rsec) { RestServerEndpoint rse = new RestServerEndpoint(rsec) {
@Override @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)); handlers.add(Tuple2.of(triggerHandler.getMessageHeaders(), triggerHandler));
return handlers; return handlers;
} }
Expand Down Expand Up @@ -280,9 +280,9 @@ public void testListJobs() throws Exception {


RestServerEndpoint rse = new RestServerEndpoint(rsec) { RestServerEndpoint rse = new RestServerEndpoint(rsec) {
@Override @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)); handlers.add(Tuple2.of(listJobsHandler.getMessageHeaders(), listJobsHandler));
return handlers; return handlers;
} }
Expand Down
Original file line number Original file line Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -137,7 +137,7 @@ public DispatcherRestEndpoint(
} }


@Override @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); ArrayList<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(3);


final Time timeout = restConfiguration.getTimeout(); final Time timeout = restConfiguration.getTimeout();
Expand Down Expand Up @@ -352,8 +352,8 @@ protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> in
handlers.add(Tuple2.of(jobSubmitHandler.getMessageHeaders(), jobSubmitHandler)); handlers.add(Tuple2.of(jobSubmitHandler.getMessageHeaders(), jobSubmitHandler));
handlers.add(Tuple2.of(TaskManagersHeaders.getInstance(), taskManagersHandler)); handlers.add(Tuple2.of(TaskManagersHeaders.getInstance(), taskManagersHandler));
handlers.add(Tuple2.of(TaskManagerDetailsHeaders.getInstance(), taskManagerDetailsHandler)); 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( optWebContent.ifPresent(
webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent))); webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent)));


Expand Down
Original file line number Original file line Diff line number Diff line change
Expand Up @@ -45,8 +45,11 @@


import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLEngine;


import java.io.Serializable;
import java.net.InetSocketAddress; 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.CompletableFuture;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;


Expand Down Expand Up @@ -88,7 +91,7 @@ public RestServerEndpoint(RestServerEndpointConfiguration configuration) {
* @param restAddressFuture future rest address of the RestServerEndpoint * @param restAddressFuture future rest address of the RestServerEndpoint
* @return Collection of AbstractRestHandler which are added to the server endpoint * @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. * Starts this REST server endpoint.
Expand All @@ -107,7 +110,23 @@ public void start() throws Exception {
final Router router = new Router(); final Router router = new Router();
final CompletableFuture<String> restAddressFuture = new CompletableFuture<>(); 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>() { ChannelInitializer<SocketChannel> initializer = new ChannelInitializer<SocketChannel>() {


Expand Down Expand Up @@ -268,4 +287,69 @@ private static void registerHandler(Router router, Tuple2<RestHandlerSpecificati
throw new RuntimeException("Unsupported http method: " + specificationHandler.f0.getHttpMethod() + '.'); 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;
}
}
}
} }
Original file line number Original file line Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List;
import java.util.Optional; import java.util.Optional;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
Expand Down Expand Up @@ -205,8 +206,8 @@ private static class TestRestServerEndpoint extends RestServerEndpoint {
} }


@Override @Override
protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) { protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
return Collections.singleton(Tuple2.of(new TestHeaders(), testHandler)); return Collections.singletonList(Tuple2.of(new TestHeaders(), testHandler));
} }
} }


Expand Down
Original file line number Original file line Diff line number Diff line change
@@ -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);
}
}

0 comments on commit 34fdf56

Please sign in to comment.