You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2017/12/07 17:50:46 UTC

aurora git commit: Update to guice 4.1.0, switch from jersey to resteasy

Repository: aurora
Updated Branches:
  refs/heads/master 2754636c6 -> 103f794ed


Update to guice 4.1.0, switch from jersey to resteasy

Upgrading guice was the original goal with this patch, which pulled along
several other dependencies.  Guice 3
[suffers from obscure errors](https://github.com/google/guice/issues/757)
when creating binding error messages with java 8 and lambdas.  This has been a
frequent source of frustration since we first upgraded to java 8 mid-2015.

I've gone spelunking down this path numerous times, and frequently hit a wall
with jersey.  We needed to upgrade jersey-guice, to upgrade jersey, to upgrade
guice.  jersey introduced their own dependency injection (HK2) in jersey 2.0,
which complicated matters.  There have been some promising developments since
(hk2-guice [bridge](https://javaee.github.io/hk2/guice-bridge.html#bi-directional-hk2-guice-bridge),
2.26 [abstracted HK2](https://jersey.github.io/release-notes/2.26.html), and
several [projects](https://github.com/Squarespace/jersey2-guice) have emerged
to solve the issue).  Unfortunately, each avenue failed with some combination
of not working well with our application design, or i just plain couldn't get
it working.  I began to look beyond jersey.

This left restlet and resteasy as the most common alternatives.  I balked early
at restlet due to their guice integration being
[apparently](https://github.com/restlet/restlet-framework-java/commits/master/modules/org.restlet.ext.guice)
[dormant](https://stackoverflow.com/questions/8227583/what-is-the-status-of-org-restlet-ext-guice).

Fortunately i achieved some early wins with resteasy!  Migrating was
straightforward with a small patch based on some examples.

However, i hit a hurdle with shiro-guice.  It
[needed to be updated](https://issues.apache.org/jira/browse/SHIRO-493) to work
with guice 4, and there were some necessary API changes.  No big deal, just the
`filterConfig()` nesting you see in this patch.  This revealed a deeper issue
with binding custom `Filter`s with `ShiroWebModule`.  Previously,
`ShiroWebModule` would effectively only `bind()` keys
[they define](https://github.com/apache/shiro/blob/f326fd3814f672464deb11c3dadadb27af618eec/support/guice/src/main/java/org/apache/shiro/guice/web/ShiroWebModule.java#L65-L86), allowing the API user to `bind()` custom keys.  The [patch](https://github.com/apache/shiro/commit/f2dfa7ff39c9870e7b9856ceca8690c5398080fa#diff-359a7b20d3b7fdcc0ffce11ad57d6e1c) to support guice 4 changed that, and `bind()` will be [called](https://github.com/apache/shiro/commit/f2dfa7ff39c9870e7b9856ceca8690c5398080fa#diff-359a7b20d3b7fdcc0ffce11ad57d6e1cR183)
on these custom keys.  In our case, this caused a duplicate binding.

The simplest workaround to this was to avoid `bind()`ing the custom
`afterAuthFilter` key, and use the custom type as the key type (e.g.
`Key.get(CountingFilter.class)` rather than `Key.get(Filter.class)`).

Lastly, `GuiceResteasyBootstrapServletContextListener` does not integrate with
`GuiceServletContextListener` in the way `GuiceFilter`
[demands](https://github.com/google/guice/blob/e7bef34ef6379735e2a58df1b23f351bb7e30e44/extensions/servlet/src/com/google/inject/servlet/ServletModule.java#L322-L323),
which necessitated the passing of `ServletContext` you see in this patch.

I can't say i'm happy with the outcome here, but i am overall happier given
that guice is upgraded.

Reviewed at https://reviews.apache.org/r/64362/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/103f794e
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/103f794e
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/103f794e

Branch: refs/heads/master
Commit: 103f794ed126e135f2fe0ff1bde04a4093413521
Parents: 2754636
Author: Bill Farner <wf...@apache.org>
Authored: Thu Dec 7 09:50:22 2017 -0800
Committer: Bill Farner <wf...@apache.org>
Committed: Thu Dec 7 09:50:22 2017 -0800

----------------------------------------------------------------------
 build.gradle                                    | 27 +++++---
 .../scheduler/http/JettyServerModule.java       | 46 +++++++------
 .../aurora/scheduler/http/api/ApiModule.java    |  4 --
 .../http/api/security/HttpSecurityModule.java   | 44 ++++++++----
 .../scheduler/http/AbstractJettyTest.java       |  8 +--
 .../aurora/scheduler/http/api/ApiBetaTest.java  |  7 +-
 .../apache/aurora/scheduler/http/api/ApiIT.java |  6 +-
 .../http/api/security/HttpSecurityIT.java       | 72 +++++++++++---------
 .../ShiroKerberosAuthenticationFilterTest.java  |  6 +-
 ...berosPermissiveAuthenticationFilterTest.java |  6 +-
 10 files changed, 131 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index fd606a2..76dcc04 100644
--- a/build.gradle
+++ b/build.gradle
@@ -81,14 +81,17 @@ For more details, please see https://issues.apache.org/jira/browse/AURORA-1169
   ext.curatorRev = '2.12.0'
   ext.gsonRev = '2.3.1'
   ext.guavaRev = '23.2-jre'
-  ext.guiceRev = '3.0'
+  ext.guiceRev = '4.1.0'
+  ext.httpclientRev = '4.5.2'
+  ext.httpcoreRev = '4.4.4'
   ext.asyncHttpclientRev = '2.0.37'
   ext.jacksonRev = '2.5.1'
-  ext.jerseyRev = '1.19'
+  ext.jaxRsRev = '2.0'
   ext.junitRev = '4.12'
   ext.logbackRev = '1.2.3'
   ext.nettyRev = '4.0.52.Final'
   ext.protobufRev = '3.3.0'
+  ext.resteasyRev = '3.1.4.Final'
   ext.servletRev = '3.1.0'
   ext.slf4jRev = '1.7.25'
   ext.stringTemplateRev = '3.2.1'
@@ -107,6 +110,10 @@ For more details, please see https://issues.apache.org/jira/browse/AURORA-1169
     // See http://forums.gradle.org/gradle/topics/shouldnt-resolutionstrategy-affect-depending-projects-transitive-dependencies
     resolutionStrategy {
       failOnVersionConflict()
+      force "com.google.inject:guice:${guiceRev}"
+      force "com.google.inject.extensions:guice-multibindings:${guiceRev}"
+      force "org.apache.httpcomponents:httpclient:${httpclientRev}"
+      force "org.apache.httpcomponents:httpcore:${httpcoreRev}"
       force "com.fasterxml.jackson.core:jackson-annotations:${jacksonRev}"
       force "com.fasterxml.jackson.core:jackson-core:${jacksonRev}"
       force "com.google.code.gson:gson:${gsonRev}"
@@ -185,9 +192,9 @@ project(':commons') {
     compile "com.google.code.gson:gson:${gsonRev}"
     compile "com.google.guava:guava:${guavaRev}"
     compile "com.google.inject:guice:${guiceRev}"
-    compile "com.sun.jersey:jersey-core:${jerseyRev}"
     compile "commons-lang:commons-lang:${commonsLangRev}"
     compile "javax.servlet:javax.servlet-api:${servletRev}"
+    compile "javax.ws.rs:javax.ws.rs-api:${jaxRsRev}"
     compile "org.antlr:stringtemplate:${stringTemplateRev}"
     compile "org.apache.zookeeper:zookeeper:${zookeeperRev}"
     compile "org.easymock:easymock:3.4"
@@ -358,7 +365,7 @@ sourceSets {
 }
 
 dependencies {
-  def shiroRev = '1.2.5'
+  def shiroRev = '1.4.0'
   def jettyDep = '9.3.11.v20160721'
 
   compile project(':api')
@@ -369,14 +376,14 @@ dependencies {
   compile "com.beust:jcommander:1.72"
   compile "com.google.inject:guice:${guiceRev}"
   compile "com.google.inject.extensions:guice-assistedinject:${guiceRev}"
+  compile "com.google.inject.extensions:guice-multibindings:${guiceRev}"
+  compile "com.google.inject.extensions:guice-servlet:${guiceRev}"
   compile "com.google.protobuf:protobuf-java:${protobufRev}"
   compile 'com.hubspot.jackson:jackson-datatype-protobuf:0.9.3'
   compile "com.fasterxml.jackson.core:jackson-core:${jacksonRev}"
-  compile "com.sun.jersey:jersey-core:${jerseyRev}"
-  compile "com.sun.jersey:jersey-json:${jerseyRev}"
-  compile "com.sun.jersey:jersey-server:${jerseyRev}"
-  compile "com.sun.jersey:jersey-servlet:${jerseyRev}"
-  compile "com.sun.jersey.contribs:jersey-guice:${jerseyRev}"
+  compile "org.jboss.resteasy:resteasy-guice:${resteasyRev}"
+  compile "org.jboss.resteasy:resteasy-jackson-provider:${resteasyRev}"
+  compile "org.jboss.resteasy:resteasy-jaxrs:${resteasyRev}"
   compile 'javax.inject:javax.inject:1'
   compile "javax.servlet:javax.servlet-api:${servletRev}"
   compile "org.antlr:stringtemplate:${stringTemplateRev}"
@@ -394,7 +401,7 @@ dependencies {
   compile "org.eclipse.jetty:jetty-servlets:${jettyDep}"
   compile 'org.quartz-scheduler:quartz:2.2.2'
 
-  testCompile "com.sun.jersey:jersey-client:${jerseyRev}"
+  testCompile 'com.sun.jersey:jersey-client:1.19'
   testCompile "junit:junit:${junitRev}"
   testCompile "org.powermock:powermock-module-junit4:1.6.4"
   testCompile "org.powermock:powermock-api-easymock:1.6.4"

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java b/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
index 0f8528c..9a03140 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
@@ -17,12 +17,15 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Function;
 
 import javax.inject.Inject;
 import javax.inject.Singleton;
 import javax.servlet.DispatcherType;
+import javax.servlet.ServletContext;
 import javax.servlet.ServletContextListener;
 import javax.ws.rs.HttpMethod;
 
@@ -34,6 +37,7 @@ import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.ImmutableSet;
@@ -49,10 +53,8 @@ import com.google.inject.Provides;
 import com.google.inject.TypeLiteral;
 import com.google.inject.name.Names;
 import com.google.inject.servlet.GuiceFilter;
-import com.google.inject.servlet.GuiceServletContextListener;
+import com.google.inject.servlet.ServletModule;
 import com.google.inject.util.Modules;
-import com.sun.jersey.guice.JerseyServletModule;
-import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 
 import org.apache.aurora.common.net.http.handlers.AbortHandler;
 import org.apache.aurora.common.net.http.handlers.ContentionPrinter;
@@ -82,13 +84,13 @@ import org.eclipse.jetty.server.handler.gzip.GzipHandler;
 import org.eclipse.jetty.servlet.DefaultServlet;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.util.resource.Resource;
+import org.jboss.resteasy.plugins.guice.GuiceResteasyBootstrapServletContextListener;
+import org.jboss.resteasy.plugins.server.servlet.HttpServletDispatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.util.Objects.requireNonNull;
 
-import static com.sun.jersey.api.json.JSONConfiguration.FEATURE_POJO_MAPPING;
-
 /**
  * Binding module for scheduler HTTP servlets.
  * <p>
@@ -121,9 +123,6 @@ public class JettyServerModule extends AbstractModule {
     public String listenIp;
   }
 
-  public static final Map<String, String> GUICE_CONTAINER_PARAMS = ImmutableMap.of(
-      FEATURE_POJO_MAPPING, Boolean.TRUE.toString());
-
   private static final String STATIC_ASSETS_ROOT = Resource
       .newClassPathResource("scheduler/assets/index.html")
       .toString()
@@ -193,9 +192,9 @@ public class JettyServerModule extends AbstractModule {
         ServletContextListener provideServletContextListener(Injector parentInjector) {
           return makeServletContextListener(
               parentInjector,
-              Modules.combine(
+              (servletContext) -> Modules.combine(
                   new ApiModule(options.api),
-                  new HttpSecurityModule(options),
+                  new HttpSecurityModule(options, servletContext),
                   new ThriftModule(),
                   new AopModule(options)));
         }
@@ -253,15 +252,15 @@ public class JettyServerModule extends AbstractModule {
   // TODO(ksweeney): Factor individual servlet configurations to their own ServletModules.
   @VisibleForTesting
   static ServletContextListener makeServletContextListener(
-      final Injector parentInjector,
-      final Module childModule) {
+      Injector parentInjector,
+      Function<ServletContext, Module> childModuleFactory) {
 
-    return new GuiceServletContextListener() {
+    ServletContextListener contextListener = new GuiceResteasyBootstrapServletContextListener() {
       @Override
-      protected Injector getInjector() {
-        return parentInjector.createChildInjector(
-            childModule,
-            new JerseyServletModule() {
+      protected List<? extends Module> getModules(ServletContext context) {
+        return ImmutableList.of(
+            childModuleFactory.apply(context),
+            new ServletModule() {
               @Override
               protected void configureServlets() {
                 bind(HttpStatsFilter.class).in(Singleton.class);
@@ -271,10 +270,6 @@ public class JettyServerModule extends AbstractModule {
                 filterRegex(allOf(LEADER_ENDPOINTS))
                     .through(LeaderRedirectFilter.class);
 
-                bind(GuiceContainer.class).in(Singleton.class);
-                filterRegex(allOf(ImmutableSet.copyOf(JAX_RS_ENDPOINTS.values())))
-                    .through(GuiceContainer.class, GUICE_CONTAINER_PARAMS);
-
                 filterRegex("/assets/scheduler(?:/.*)?").through(LeaderRedirectFilter.class);
 
                 serve("/assets", "/assets/*")
@@ -287,9 +282,15 @@ public class JettyServerModule extends AbstractModule {
                   bind(jaxRsHandler);
                 }
               }
-            });
+            }
+        );
       }
     };
+
+    // Injects the Injector into GuiceResteasyBootstrapServletContextListener.
+    parentInjector.injectMembers(contextListener);
+
+    return contextListener;
   }
 
   static class RedirectMonitor extends AbstractIdleService {
@@ -380,6 +381,7 @@ public class JettyServerModule extends AbstractModule {
       servletHandler.addServlet(DefaultServlet.class, "/");
       servletHandler.addFilter(GuiceFilter.class, "/*", EnumSet.allOf(DispatcherType.class));
       servletHandler.addEventListener(servletContextListener);
+      servletHandler.addServlet(HttpServletDispatcher.class, "/*");
 
       HandlerCollection rootHandler = new HandlerList();
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java b/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java
index a19663a..91ff8d3 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java
@@ -21,11 +21,9 @@ import com.beust.jcommander.Parameters;
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Provides;
 import com.google.inject.servlet.ServletModule;
-import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 
 import org.apache.aurora.gen.AuroraAdmin;
 import org.apache.aurora.scheduler.http.CorsFilter;
-import org.apache.aurora.scheduler.http.JettyServerModule;
 import org.apache.aurora.scheduler.http.LeaderRedirectFilter;
 import org.apache.aurora.scheduler.http.api.TContentAwareServlet.ContentFactoryPair;
 import org.apache.aurora.scheduler.http.api.TContentAwareServlet.InputConfig;
@@ -75,8 +73,6 @@ public class ApiModule extends ServletModule {
     serve(API_PATH).with(TContentAwareServlet.class);
 
     filter(ApiBeta.PATH, ApiBeta.PATH + "/*").through(LeaderRedirectFilter.class);
-    filter(ApiBeta.PATH, ApiBeta.PATH + "/*")
-        .through(GuiceContainer.class, JettyServerModule.GUICE_CONTAINER_PARAMS);
     bind(ApiBeta.class);
 
     serve("/apiclient", "/apiclient/*")

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java b/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java
index d81671c..9d757db 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java
@@ -19,6 +19,7 @@ import java.util.Optional;
 import java.util.Set;
 
 import javax.servlet.Filter;
+import javax.servlet.ServletContext;
 
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
@@ -118,32 +119,38 @@ public class HttpSecurityModule extends ServletModule {
   private final HttpAuthenticationMechanism mechanism;
   private final Set<Module> shiroConfigurationModules;
   private final Optional<Key<? extends Filter>> shiroAfterAuthFilterKey;
+  private final ServletContext servletContext;
 
-  public HttpSecurityModule(CliOptions options) {
+  public HttpSecurityModule(CliOptions options, ServletContext servletContext) {
     this(
         options.httpSecurity.httpAuthenticationMechanism,
         MoreModules.instantiateAll(options.httpSecurity.shiroRealmModule, options),
-        Optional.ofNullable(options.httpSecurity.shiroAfterAuthFilter).map(Key::get).orElse(null));
+        Optional.ofNullable(options.httpSecurity.shiroAfterAuthFilter).map(Key::get).orElse(null),
+        servletContext);
   }
 
   @VisibleForTesting
   HttpSecurityModule(
       Module shiroConfigurationModule,
-      Key<? extends Filter> shiroAfterAuthFilterKey) {
+      Key<? extends Filter> shiroAfterAuthFilterKey,
+      ServletContext servletContext) {
 
     this(HttpAuthenticationMechanism.BASIC,
         ImmutableSet.of(shiroConfigurationModule),
-        shiroAfterAuthFilterKey);
+        shiroAfterAuthFilterKey,
+        servletContext);
   }
 
   private HttpSecurityModule(
       HttpAuthenticationMechanism mechanism,
       Set<Module> shiroConfigurationModules,
-      Key<? extends Filter> shiroAfterAuthFilterKey) {
+      Key<? extends Filter> shiroAfterAuthFilterKey,
+      ServletContext servletContext) {
 
     this.mechanism = requireNonNull(mechanism);
     this.shiroConfigurationModules = requireNonNull(shiroConfigurationModules);
     this.shiroAfterAuthFilterKey = Optional.ofNullable(shiroAfterAuthFilterKey);
+    this.servletContext = requireNonNull(servletContext);
   }
 
   @Override
@@ -171,7 +178,7 @@ public class HttpSecurityModule extends ServletModule {
       }
     });
     install(guiceFilterModule(API_PATH));
-    install(new ShiroWebModule(getServletContext()) {
+    install(new ShiroWebModule(servletContext) {
 
       // Replace the ServletContainerSessionManager which causes subject.runAs(...) in a
       // downstream user-defined filter to fail. See also: SHIRO-554
@@ -193,11 +200,11 @@ public class HttpSecurityModule extends ServletModule {
         // more specific pattern first.
         switch (mechanism) {
           case BASIC:
-            addFilterChainWithAfterAuthFilter(config(AUTHC_BASIC, PERMISSIVE));
+            addFilterChainWithAfterAuthFilter(filterConfig(AUTHC_BASIC, PERMISSIVE));
             break;
 
           case NEGOTIATE:
-            addFilterChainWithAfterAuthFilter(K_PERMISSIVE);
+            addFilterChainWithAfterAuthFilter(filterConfig(K_PERMISSIVE));
             break;
 
           default:
@@ -206,22 +213,31 @@ public class HttpSecurityModule extends ServletModule {
         }
       }
 
-      private void addFilterChainWithAfterAuthFilter(Key<? extends Filter> filter) {
+      private void addFilterChainWithAfterAuthFilter(FilterConfig<? extends Filter> filter) {
         if (shiroAfterAuthFilterKey.isPresent()) {
-          addFilterChain(filter, shiroAfterAuthFilterKey.get());
+          addFilterChain(filter, filterConfig(shiroAfterAuthFilterKey.get()));
         } else {
           addFilterChain(filter);
         }
       }
 
       @SuppressWarnings("unchecked")
-      private void addFilterChain(Key<? extends Filter> filter) {
-        addFilterChain(ALL_PATTERN, NO_SESSION_CREATION, filter);
+      private void addFilterChain(FilterConfig<? extends Filter> filter) {
+        addFilterChain(
+            ALL_PATTERN,
+            filterConfig(NO_SESSION_CREATION),
+            filter);
       }
 
       @SuppressWarnings("unchecked")
-      private void addFilterChain(Key<? extends Filter> filter1, Key<? extends Filter> filter2) {
-        addFilterChain(ALL_PATTERN, NO_SESSION_CREATION, filter1, filter2);
+      private void addFilterChain(
+          FilterConfig<? extends Filter> filter1,
+          FilterConfig<? extends Filter> filter2) {
+        addFilterChain(
+            ALL_PATTERN,
+            filterConfig(NO_SESSION_CREATION),
+            filter1,
+            filter2);
       }
     });
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java b/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java
index a3f6941..f04e11a 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java
@@ -15,7 +15,9 @@ package org.apache.aurora.scheduler.http;
 
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
 
+import javax.servlet.ServletContext;
 import javax.servlet.ServletContextListener;
 import javax.ws.rs.core.MediaType;
 
@@ -34,7 +36,6 @@ import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.client.config.ClientConfig;
 import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.api.json.JSONConfiguration;
 
 import org.apache.aurora.GuavaUtils.ServiceManagerIface;
 import org.apache.aurora.common.quantity.Amount;
@@ -88,8 +89,8 @@ public abstract class AbstractJettyTest extends EasyMockTest {
    *
    * @return A module used in the creation of the servlet container's child injector.
    */
-  protected Module getChildServletModule() {
-    return Modules.EMPTY_MODULE;
+  protected Function<ServletContext, Module> getChildServletModule() {
+    return (servletContext) -> Modules.EMPTY_MODULE;
   }
 
   @Before
@@ -189,7 +190,6 @@ public abstract class AbstractJettyTest extends EasyMockTest {
   protected WebResource.Builder getRequestBuilder(String path) {
     assertNotNull("HTTP server must be started first", httpServer);
     ClientConfig config = new DefaultClientConfig();
-    config.getFeatures().put(JSONConfiguration.FEATURE_POJO_MAPPING, Boolean.TRUE);
     config.getClasses().add(GsonMessageBodyHandler.class);
     Client client = Client.create(config);
     // Disable redirects so we can unit test them.

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/test/java/org/apache/aurora/scheduler/http/api/ApiBetaTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/ApiBetaTest.java b/src/test/java/org/apache/aurora/scheduler/http/api/ApiBetaTest.java
index e8ef6bd..530bfa2 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/ApiBetaTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/ApiBetaTest.java
@@ -13,6 +13,9 @@
  */
 package org.apache.aurora.scheduler.http.api;
 
+import java.util.function.Function;
+
+import javax.servlet.ServletContext;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 
@@ -64,8 +67,8 @@ public class ApiBetaTest extends AbstractJettyTest {
   }
 
   @Override
-  protected Module getChildServletModule() {
-    return Modules.combine(
+  protected Function<ServletContext, Module> getChildServletModule() {
+    return (servletContext) -> Modules.combine(
         new ApiModule(new ApiModule.Options()),
         new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java b/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java
index 43fa315..10da43b 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java
@@ -15,7 +15,9 @@ package org.apache.aurora.scheduler.http.api;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.function.Function;
 
+import javax.servlet.ServletContext;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.MediaType;
 
@@ -49,8 +51,8 @@ public class ApiIT extends AbstractJettyTest {
   }
 
   @Override
-  protected Module getChildServletModule() {
-    return Modules.combine(
+  protected Function<ServletContext, Module> getChildServletModule() {
+    return (servletContext) -> Modules.combine(
         new ApiModule(new ApiModule.Options()),
         new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
index d3c7ac9..a6e0e57 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
@@ -15,12 +15,15 @@ package org.apache.aurora.scheduler.http.api.security;
 
 import java.io.IOException;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
 
+import javax.inject.Inject;
 import javax.servlet.Filter;
-import javax.servlet.FilterChain;
+import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableSet;
@@ -53,21 +56,19 @@ import org.apache.shiro.authc.credential.CredentialsMatcher;
 import org.apache.shiro.authc.credential.SimpleCredentialsMatcher;
 import org.apache.shiro.config.Ini;
 import org.apache.shiro.realm.text.IniRealm;
+import org.apache.shiro.web.filter.PathMatchingFilter;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TJSONProtocol;
 import org.apache.thrift.transport.THttpClient;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
-import org.easymock.IExpectationSetters;
 import org.junit.Before;
 import org.junit.Test;
 
 import static org.apache.aurora.scheduler.http.api.ApiModule.API_PATH;
 import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.getCurrentArguments;
-import static org.easymock.EasyMock.isA;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class HttpSecurityIT extends AbstractJettyTest {
@@ -107,7 +108,7 @@ public class HttpSecurityIT extends AbstractJettyTest {
   private Ini ini;
   private Class<? extends CredentialsMatcher> credentialsMatcher;
   private AnnotatedAuroraAdmin auroraAdmin;
-  private Filter shiroAfterAuthFilter;
+  private AtomicInteger afterAuthCalls;
 
   @Before
   public void setUp() {
@@ -139,22 +140,42 @@ public class HttpSecurityIT extends AbstractJettyTest {
             + ADS_STAGING_JOB.getName());
 
     auroraAdmin = createMock(AnnotatedAuroraAdmin.class);
-    shiroAfterAuthFilter = createMock(Filter.class);
+    afterAuthCalls = new AtomicInteger();
+  }
+
+  public static class CountingFilter extends PathMatchingFilter {
+    private final AtomicInteger calls;
+
+    @Inject
+    public CountingFilter(AtomicInteger calls) {
+      this.calls = calls;
+    }
+
+    @Override
+    protected boolean onPreHandle(
+        ServletRequest request,
+        ServletResponse response,
+        Object mappedValue) {
+
+      calls.incrementAndGet();
+      return true;
+    }
   }
 
   @Override
-  protected Module getChildServletModule() {
-    return Modules.combine(
+  protected Function<ServletContext, Module> getChildServletModule() {
+    Key<? extends Filter> afterAuthBinding =
+        Key.get(CountingFilter.class, SHIRO_AFTER_AUTH_FILTER_ANNOTATION);
+    return (servletContext) -> Modules.combine(
         new ApiModule(new ApiModule.Options()),
         new HttpSecurityModule(
             new IniShiroRealmModule(ini, credentialsMatcher),
-            Key.get(Filter.class, SHIRO_AFTER_AUTH_FILTER_ANNOTATION)),
+            afterAuthBinding,
+            servletContext),
         new AbstractModule() {
           @Override
           protected void configure() {
-            bind(Filter.class)
-                .annotatedWith(SHIRO_AFTER_AUTH_FILTER_ANNOTATION)
-                .toInstance(shiroAfterAuthFilter);
+            bind(AtomicInteger.class).toInstance(afterAuthCalls);
             MockDecoratedThrift.bindForwardedMock(binder(), auroraAdmin);
           }
         });
@@ -186,25 +207,9 @@ public class HttpSecurityIT extends AbstractJettyTest {
     return getClient(defaultHttpClient);
   }
 
-  private IExpectationSetters<Object> expectShiroAfterAuthFilter()
-      throws ServletException, IOException {
-
-    shiroAfterAuthFilter.doFilter(
-        isA(HttpServletRequest.class),
-        isA(HttpServletResponse.class),
-        isA(FilterChain.class));
-
-    return expectLastCall().andAnswer(() -> {
-      Object[] args = getCurrentArguments();
-      ((FilterChain) args[2]).doFilter((HttpServletRequest) args[0], (HttpServletResponse) args[1]);
-      return null;
-    });
-  }
-
   @Test
   public void testReadOnlyScheduler() throws TException, ServletException, IOException {
     expect(auroraAdmin.getRoleSummary()).andReturn(OK).times(3);
-    expectShiroAfterAuthFilter().times(3);
 
     replayAndStart();
 
@@ -213,6 +218,7 @@ public class HttpSecurityIT extends AbstractJettyTest {
     // Incorrect works because the server doesn't challenge for credentials to execute read-only
     // methods.
     assertEquals(OK, getAuthenticatedClient(INCORRECT).getRoleSummary());
+    assertEquals(3, afterAuthCalls.get());
   }
 
   private void assertKillTasksFails(AuroraAdmin.Client client) throws TException {
@@ -230,7 +236,6 @@ public class HttpSecurityIT extends AbstractJettyTest {
 
     expect(auroraAdmin.killTasks(job, null, null)).andReturn(OK).times(2);
     expect(auroraAdmin.killTasks(ADS_STAGING_JOB.newBuilder(), null, null)).andReturn(OK);
-    expectShiroAfterAuthFilter().atLeastOnce();
 
     replayAndStart();
 
@@ -272,6 +277,7 @@ public class HttpSecurityIT extends AbstractJettyTest {
     assertKillTasksFails(getUnauthenticatedClient());
     assertKillTasksFails(getAuthenticatedClient(INCORRECT));
     assertKillTasksFails(getAuthenticatedClient(NONEXISTENT));
+    assertTrue(afterAuthCalls.get() > 0);
   }
 
   private void assertSnapshotFails(AuroraAdmin.Client client) throws TException {
@@ -287,7 +293,6 @@ public class HttpSecurityIT extends AbstractJettyTest {
   public void testAuroraAdmin() throws TException, ServletException, IOException {
     expect(auroraAdmin.snapshot()).andReturn(OK);
     expect(auroraAdmin.listBackups()).andReturn(OK);
-    expectShiroAfterAuthFilter().times(12);
 
     replayAndStart();
 
@@ -304,5 +309,6 @@ public class HttpSecurityIT extends AbstractJettyTest {
     }
 
     assertEquals(OK, getAuthenticatedClient(BACKUP_SERVICE).listBackups());
+    assertEquals(12, afterAuthCalls.get());
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosAuthenticationFilterTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosAuthenticationFilterTest.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosAuthenticationFilterTest.java
index 20e3d62..1ffc72b 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosAuthenticationFilterTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosAuthenticationFilterTest.java
@@ -14,7 +14,9 @@
 package org.apache.aurora.scheduler.http.api.security;
 
 import java.io.IOException;
+import java.util.function.Function;
 
+import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
@@ -59,8 +61,8 @@ public class ShiroKerberosAuthenticationFilterTest extends AbstractJettyTest {
   }
 
   @Override
-  public Module getChildServletModule() {
-    return new ServletModule() {
+  public Function<ServletContext, Module> getChildServletModule() {
+    return (servletContext) -> new ServletModule() {
       @Override
       protected void configureServlets() {
         filter(PATH).through(filter);

http://git-wip-us.apache.org/repos/asf/aurora/blob/103f794e/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosPermissiveAuthenticationFilterTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosPermissiveAuthenticationFilterTest.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosPermissiveAuthenticationFilterTest.java
index 0018467..4ede727 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosPermissiveAuthenticationFilterTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroKerberosPermissiveAuthenticationFilterTest.java
@@ -14,7 +14,9 @@
 package org.apache.aurora.scheduler.http.api.security;
 
 import java.io.IOException;
+import java.util.function.Function;
 
+import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
@@ -52,8 +54,8 @@ public class ShiroKerberosPermissiveAuthenticationFilterTest extends AbstractJet
   }
 
   @Override
-  public Module getChildServletModule() {
-    return new ServletModule() {
+  public Function<ServletContext, Module> getChildServletModule() {
+    return (servletContext) -> new ServletModule() {
       @Override
       protected void configureServlets() {
         filter(PATH).through(filter);