You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ke...@apache.org on 2015/04/22 20:20:44 UTC

aurora git commit: Add typed Shiro permissions SPI.

Repository: aurora
Updated Branches:
  refs/heads/master 8fd21a1ad -> 352e0ef55


Add typed Shiro permissions SPI.

Working on an implementation of a Shiro Realm to replace the old
CapabilityValidator I realized we're missing some information with
the new API. This patch allows a Realm implementation to optionally
introspect a permission check for Aurora-specific information about
exactly what's being attempted, while maintaining compatibility
with Shiro realms that don't know anything about Aurora, such as the
included IniRealm.

To do this I've added a new SPI package, and documented
backwards-compatibility considerations with it. The idea is that
a third party can write an Aurora-aware Shiro Realm module against
a stable ABI and gets one release to update it if we want to make
changes to it.

Testing Done:
./gradlew -Pq build

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


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

Branch: refs/heads/master
Commit: 352e0ef55f40477cbc55ed21abbdf8738fb82df4
Parents: 8fd21a1
Author: Kevin Sweeney <ke...@apache.org>
Authored: Wed Apr 22 11:20:02 2015 -0700
Committer: Kevin Sweeney <ke...@apache.org>
Committed: Wed Apr 22 11:20:02 2015 -0700

----------------------------------------------------------------------
 config/findbugs/excludeFilter.xml               |  12 ++
 .../http/api/security/ApiSecurityModule.java    |  19 +-
 .../security/ShiroAuthorizingInterceptor.java   |  16 +-
 .../ShiroAuthorizingParamInterceptor.java       |  30 +--
 .../aurora/scheduler/spi/Permissions.java       | 199 +++++++++++++++++++
 .../aurora/scheduler/spi/package-info.java      |  22 ++
 .../ShiroAuthorizingInterceptorTest.java        |  10 +-
 .../ShiroAuthorizingParamInterceptorTest.java   |   5 +-
 8 files changed, 265 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/352e0ef5/config/findbugs/excludeFilter.xml
----------------------------------------------------------------------
diff --git a/config/findbugs/excludeFilter.xml b/config/findbugs/excludeFilter.xml
index 0bff71c..328e75c 100644
--- a/config/findbugs/excludeFilter.xml
+++ b/config/findbugs/excludeFilter.xml
@@ -43,6 +43,18 @@ limitations under the License.
     <Bug pattern="SE_BAD_FIELD" />
   </Match>
 
+  <!-- We must maintain reflexivity with WildcardPermission's equals, so we cannot override
+       it even though we add fields in our subclass. We cannot use delegation because
+       WildcardPermission's implies method checks that its argument is an instance of
+       WildcardPermission. -->
+  <Match>
+    <Or>
+      <Class name="org.apache.aurora.scheduler.spi.Permissions$JobScopedRpcPermission" />
+      <Class name="org.apache.aurora.scheduler.spi.Permissions$UnscopedRpcPermission" />
+    </Or>
+    <Bug pattern="EQ_DOESNT_OVERRIDE_EQUALS" />
+  </Match>
+
   <!-- Method is intentionally only callable by EventBus. -->
   <Match>
     <Class name="org.apache.aurora.scheduler.events.PubsubEventModule$1" />

http://git-wip-us.apache.org/repos/asf/aurora/blob/352e0ef5/src/main/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityModule.java b/src/main/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityModule.java
index 0265e2a..ca8e23b 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityModule.java
@@ -44,6 +44,9 @@ import org.apache.shiro.web.filter.authc.BasicHttpAuthenticationFilter;
 
 import static java.util.Objects.requireNonNull;
 
+import static org.apache.aurora.scheduler.spi.Permissions.Domain.THRIFT_AURORA_ADMIN;
+import static org.apache.aurora.scheduler.spi.Permissions.Domain.THRIFT_AURORA_SCHEDULER_MANAGER;
+
 /**
  * Provides HTTP Basic Authentication for the API using Apache Shiro. When enabled, prevents
  * unauthenticated access to write APIs. Write API access must also be authorized, with permissions
@@ -51,16 +54,6 @@ import static java.util.Objects.requireNonNull;
  * this package.
  */
 public class ApiSecurityModule extends ServletModule {
-  /**
-   * Prefix for the permission protecting all AuroraSchedulerManager RPCs.
-   */
-  public static final String AURORA_SCHEDULER_MANAGER_PERMISSION = "thrift.AuroraSchedulerManager";
-
-  /**
-   * Prefix for the permission protecting all AuroraAdmin RPCs.
-   */
-  public static final String AURORA_ADMIN_PERMISSION = "thrift.AuroraAdmin";
-
   public static final String HTTP_REALM_NAME = "Apache Aurora Scheduler";
 
   @CmdLine(name = "enable_api_security",
@@ -168,15 +161,15 @@ public class ApiSecurityModule extends ServletModule {
         AURORA_SCHEDULER_MANAGER_SERVICE.or(AURORA_ADMIN_SERVICE),
         authenticatingInterceptor);
 
-    MethodInterceptor apiInterceptor =
-        new ShiroAuthorizingParamInterceptor(AURORA_SCHEDULER_MANAGER_PERMISSION);
+    MethodInterceptor apiInterceptor = new ShiroAuthorizingParamInterceptor(
+        THRIFT_AURORA_SCHEDULER_MANAGER);
     requestInjection(apiInterceptor);
     bindInterceptor(
         Matchers.subclassesOf(AuroraSchedulerManager.Iface.class),
         AURORA_SCHEDULER_MANAGER_SERVICE,
         apiInterceptor);
 
-    MethodInterceptor adminInterceptor = new ShiroAuthorizingInterceptor(AURORA_ADMIN_PERMISSION);
+    MethodInterceptor adminInterceptor = new ShiroAuthorizingInterceptor(THRIFT_AURORA_ADMIN);
     requestInjection(adminInterceptor);
     bindInterceptor(
         Matchers.subclassesOf(AnnotatedAuroraAdmin.class),

http://git-wip-us.apache.org/repos/asf/aurora/blob/352e0ef5/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptor.java b/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptor.java
index 7a124cc..98bc434 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptor.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptor.java
@@ -21,17 +21,16 @@ import javax.inject.Inject;
 import javax.inject.Provider;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.twitter.common.base.MorePreconditions;
 import com.twitter.common.stats.StatsProvider;
 
 import org.aopalliance.intercept.MethodInterceptor;
 import org.aopalliance.intercept.MethodInvocation;
 import org.apache.aurora.gen.Response;
 import org.apache.aurora.gen.ResponseCode;
+import org.apache.aurora.scheduler.spi.Permissions;
+import org.apache.aurora.scheduler.spi.Permissions.Domain;
 import org.apache.aurora.scheduler.thrift.Responses;
 import org.apache.shiro.authz.Permission;
-import org.apache.shiro.authz.permission.WildcardPermission;
 import org.apache.shiro.subject.Subject;
 
 import static java.util.Objects.requireNonNull;
@@ -54,17 +53,15 @@ class ShiroAuthorizingInterceptor implements MethodInterceptor {
   @VisibleForTesting
   static final String SHIRO_AUTHORIZATION_FAILURES = "shiro_authorization_failures";
 
-  private static final Joiner PERMISSION_JOINER = Joiner.on(":");
-
-  private final String permissionPrefix;
+  private final Domain domain;
 
   private volatile boolean initialized;
 
   private Provider<Subject> subjectProvider;
   private AtomicLong shiroAdminAuthorizationFailures;
 
-  ShiroAuthorizingInterceptor(String permissionPrefix) {
-    this.permissionPrefix = MorePreconditions.checkNotBlank(permissionPrefix);
+  ShiroAuthorizingInterceptor(Domain domain) {
+    this.domain = requireNonNull(domain);
   }
 
   @Inject
@@ -84,8 +81,7 @@ class ShiroAuthorizingInterceptor implements MethodInterceptor {
     checkArgument(Response.class.isAssignableFrom(method.getReturnType()));
 
     Subject subject = subjectProvider.get();
-    Permission checkedPermission = new WildcardPermission(
-        PERMISSION_JOINER.join(permissionPrefix, method.getName()));
+    Permission checkedPermission = Permissions.createUnscopedPermission(domain, method.getName());
     if (subject.isPermitted(checkedPermission)) {
       return invocation.proceed();
     } else {

http://git-wip-us.apache.org/repos/asf/aurora/blob/352e0ef5/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java b/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java
index fde6c84..7425b93 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java
@@ -17,7 +17,6 @@ import java.lang.reflect.Method;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -26,7 +25,6 @@ import javax.inject.Provider;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
-import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
@@ -40,7 +38,6 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.reflect.Invokable;
 import com.google.common.reflect.Parameter;
-import com.twitter.common.base.MorePreconditions;
 import com.twitter.common.stats.StatsProvider;
 
 import org.aopalliance.intercept.MethodInterceptor;
@@ -60,13 +57,16 @@ import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.http.api.security.FieldGetter.AbstractFieldGetter;
 import org.apache.aurora.scheduler.http.api.security.FieldGetter.IdentityFieldGetter;
+import org.apache.aurora.scheduler.spi.Permissions;
+import org.apache.aurora.scheduler.spi.Permissions.Domain;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.thrift.Responses;
 import org.apache.shiro.authz.Permission;
-import org.apache.shiro.authz.permission.WildcardPermission;
 import org.apache.shiro.subject.Subject;
 import org.apache.thrift.TBase;
 
+import static java.util.Objects.requireNonNull;
+
 import static com.google.common.base.Preconditions.checkState;
 
 /**
@@ -285,27 +285,26 @@ class ShiroAuthorizingParamInterceptor implements MethodInterceptor {
         }
       };
 
-  private static final Joiner COLON_JOINER = Joiner.on(":");
-
   private final LoadingCache<Method, Function<Object[], Optional<JobKey>>> authorizingParamGetters =
       CacheBuilder.<Method, Function<Object[], Optional<JobKey>>>newBuilder().build(LOADER);
 
-  private final String permissionPrefix;
+  private final Domain domain;
+
   private volatile boolean initialized;
 
   private Provider<Subject> subjectProvider;
   private AtomicLong authorizationFailures;
   private AtomicLong badRequests;
 
-  ShiroAuthorizingParamInterceptor(String permissionPrefix) {
-    this.permissionPrefix = MorePreconditions.checkNotBlank(permissionPrefix);
+  ShiroAuthorizingParamInterceptor(Domain domain) {
+    this.domain = requireNonNull(domain);
   }
 
   @Inject
   void initialize(Provider<Subject> newSubjectProvider, StatsProvider statsProvider) {
     checkState(!initialized);
 
-    this.subjectProvider = Objects.requireNonNull(newSubjectProvider);
+    this.subjectProvider = requireNonNull(newSubjectProvider);
     authorizationFailures = statsProvider.makeCounter(SHIRO_AUTHORIZATION_FAILURES);
     badRequests = statsProvider.makeCounter(SHIRO_BAD_REQUESTS);
 
@@ -314,19 +313,12 @@ class ShiroAuthorizingParamInterceptor implements MethodInterceptor {
 
   @VisibleForTesting
   Permission makeWildcardPermission(String methodName) {
-    return new WildcardPermission(
-        COLON_JOINER.join(permissionPrefix, methodName));
+    return Permissions.createUnscopedPermission(domain, methodName);
   }
 
   @VisibleForTesting
   Permission makeTargetPermission(String methodName, IJobKey jobKey) {
-    return new WildcardPermission(
-        COLON_JOINER.join(
-            permissionPrefix,
-            methodName,
-            jobKey.getRole(),
-            jobKey.getEnvironment(),
-            jobKey.getName()));
+    return Permissions.createJobScopedPermission(methodName, jobKey);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/aurora/blob/352e0ef5/src/main/java/org/apache/aurora/scheduler/spi/Permissions.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/spi/Permissions.java b/src/main/java/org/apache/aurora/scheduler/spi/Permissions.java
new file mode 100644
index 0000000..b9dff44
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/spi/Permissions.java
@@ -0,0 +1,199 @@
+/**
+ * Licensed 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.aurora.scheduler.spi;
+
+import com.google.common.base.Optional;
+
+import org.apache.aurora.scheduler.storage.entities.IJobKey;
+import org.apache.shiro.authz.Permission;
+import org.apache.shiro.authz.permission.WildcardPermission;
+
+import static java.util.Objects.requireNonNull;
+
+import static com.google.common.base.Objects.toStringHelper;
+
+/**
+ * Utilities for Aurora-specific Shiro permissions.
+ */
+public final class Permissions {
+  private Permissions() {
+    // Utility class.
+  }
+
+  /**
+   * A permission representing an intended invocation of an RPC exposed by the Aurora scheduler API.
+   *
+   * Drop-in implementations of {@link org.apache.shiro.realm.Realm} that construct standard Shiro
+   * {@link org.apache.shiro.authz.permission.WildcardPermission}s will work fine, but realms that
+   * want to authorize or account access using type-safe Aurora-specific information can inspect
+   * permission checks and add additional information if they are instances of this or its
+   * public subclasses in the SPI.
+   */
+  public interface AuroraRpcPermission extends Permission {
+    /**
+     * The domain of the RPC permitted to be invoked.
+     */
+    Domain getDomain();
+
+    /**
+     * The name of the RPC permitted to be invoked.
+     */
+    String getRpc();
+  }
+
+  /**
+   * Domain of a permitted RPC (the first part of a {@link WildcardPermission}).
+   */
+  public static enum Domain {
+    /**
+     * RPCs on the {@link org.apache.aurora.gen.AuroraSchedulerManager} service.
+     */
+    THRIFT_AURORA_SCHEDULER_MANAGER("thrift.AuroraSchedulerManager"),
+
+    /**
+     * RPCs on the {@link org.apache.aurora.gen.AuroraAdmin} service.
+     */
+    THRIFT_AURORA_ADMIN("thrift.AuroraAdmin");
+
+    private final String permissionPart;
+
+    Domain(String permissionPart) {
+      this.permissionPart = permissionPart;
+    }
+
+    /**
+     * The String form of the permission part represented by this domain.
+     */
+    @Override
+    public String toString() {
+      return permissionPart;
+    }
+
+    /**
+     * Get the {@link Domain} associated with a given permission part. Inverse of {@link #toString}.
+     *
+     * @param permissionPart The permission part representing the domain.
+     * @return The domain represented by it, if one exists.
+     */
+    public static Optional<Domain> fromString(String permissionPart) {
+      for (Domain domain : Domain.values()) {
+        if (domain.permissionPart.equals(permissionPart)) {
+          return Optional.of(domain);
+        }
+      }
+
+      return Optional.absent();
+    }
+  }
+
+  /**
+   * A permission to invoke an RPC with any arguments.
+   */
+  public static final class UnscopedRpcPermission extends WildcardPermission
+      implements AuroraRpcPermission {
+
+    private final Domain domain;
+    private final String rpc;
+
+    UnscopedRpcPermission(Domain domain, String rpc) {
+      this.domain = requireNonNull(domain);
+      this.rpc = requireNonNull(rpc);
+      setParts(String.format("%s:%s", domain, rpc));
+    }
+
+    @Override
+    public Domain getDomain() {
+      return domain;
+    }
+
+    @Override
+    public String getRpc() {
+      return rpc;
+    }
+
+    @Override
+    public String toString() {
+      return toStringHelper(this).add("domain", domain).add("rpc", rpc).toString();
+    }
+  }
+
+  /**
+   * Permission to invoke an RPC only with arguments scoped to a single job.
+   */
+  public static final class JobScopedRpcPermission extends WildcardPermission
+      implements AuroraRpcPermission {
+
+    private static final Domain DOMAIN = Domain.THRIFT_AURORA_SCHEDULER_MANAGER;
+
+    private final String rpc;
+    private final IJobKey permittedJob;
+
+    JobScopedRpcPermission(String rpc, IJobKey permittedJob) {
+      this.rpc = requireNonNull(rpc);
+      this.permittedJob = requireNonNull(permittedJob);
+
+      setParts(
+          String.format("%s:%s:%s:%s:%s",
+              DOMAIN,
+              rpc,
+              permittedJob.getRole(),
+              permittedJob.getEnvironment(),
+              permittedJob.getName()));
+    }
+
+    /**
+     * The job permitted as an argument to the permitted RPC.
+     */
+    public IJobKey getPermittedJob() {
+      return permittedJob;
+    }
+
+    @Override
+    public Domain getDomain() {
+      return DOMAIN;
+    }
+
+    @Override
+    public String getRpc() {
+      return rpc;
+    }
+
+    @Override
+    public String toString() {
+      return toStringHelper(this).add("rpc", rpc).add("permittedJob", permittedJob).toString();
+    }
+  }
+
+  /**
+   * Creates a permission permitting the given RPC to operate on a single given job.
+   *
+   * @param rpc The RPC permitted to be called.
+   * @param targetJob The job permitted to be operated upon.
+   * @return A permission permitting the given RPC to operate on the given job.
+   */
+  public static JobScopedRpcPermission createJobScopedPermission(String rpc, IJobKey targetJob) {
+    return new JobScopedRpcPermission(rpc, targetJob);
+  }
+
+  /**
+   * Creates a permission permitting invocation of the given RPC with any possible argument.
+   *
+   * @param domain The domain of the RPC.
+   * @param rpc The RPC permitted to be called.
+   * @return A permission permitting invocation of the given RPC for all arguments.
+   */
+  public static UnscopedRpcPermission createUnscopedPermission(Domain domain, String rpc) {
+    return new UnscopedRpcPermission(domain, rpc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/352e0ef5/src/main/java/org/apache/aurora/scheduler/spi/package-info.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/spi/package-info.java b/src/main/java/org/apache/aurora/scheduler/spi/package-info.java
new file mode 100644
index 0000000..9689318
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/spi/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed 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.
+ */
+/**
+ * Service Provider Interface for Aurora. Java classes outside this package, even public ones,
+ * will not necessarily be backwards ABI-compatible across releases unless they are explicitly
+ * documented as being part of a public API or SPI.
+ */
+@ParametersAreNonnullByDefault
+package org.apache.aurora.scheduler.spi;
+
+import javax.annotation.ParametersAreNonnullByDefault;

http://git-wip-us.apache.org/repos/asf/aurora/blob/352e0ef5/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptorTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptorTest.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptorTest.java
index 16f2da5..39d2df0 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptorTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingInterceptorTest.java
@@ -25,20 +25,22 @@ import org.apache.aurora.gen.AuroraAdmin;
 import org.apache.aurora.gen.Response;
 import org.apache.aurora.gen.ResponseCode;
 import org.apache.aurora.gen.SessionKey;
+import org.apache.aurora.scheduler.spi.Permissions;
+import org.apache.aurora.scheduler.spi.Permissions.Domain;
 import org.apache.aurora.scheduler.thrift.Responses;
-import org.apache.shiro.authz.permission.WildcardPermission;
 import org.apache.shiro.subject.Subject;
 import org.easymock.IExpectationSetters;
 import org.junit.Before;
 import org.junit.Test;
 
 import static org.apache.aurora.scheduler.http.api.security.ShiroAuthorizingInterceptor.SHIRO_AUTHORIZATION_FAILURES;
+import static org.apache.aurora.scheduler.spi.Permissions.Domain.THRIFT_AURORA_ADMIN;
 import static org.easymock.EasyMock.expect;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertSame;
 
 public class ShiroAuthorizingInterceptorTest extends EasyMockTest {
-  private static final String PERMISSION_PREFIX = "adminRPC";
+  private static final Domain DOMAIN = THRIFT_AURORA_ADMIN;
 
   private Subject subject;
   private StatsProvider statsProvider;
@@ -49,7 +51,7 @@ public class ShiroAuthorizingInterceptorTest extends EasyMockTest {
 
   @Before
   public void setUp() throws NoSuchMethodException {
-    interceptor = new ShiroAuthorizingInterceptor(PERMISSION_PREFIX);
+    interceptor = new ShiroAuthorizingInterceptor(DOMAIN);
     subject = createMock(Subject.class);
     statsProvider = createMock(StatsProvider.class);
     methodInvocation = createMock(MethodInvocation.class);
@@ -64,7 +66,7 @@ public class ShiroAuthorizingInterceptorTest extends EasyMockTest {
 
   private IExpectationSetters<Boolean> expectSubjectPermitted() {
     return expect(subject.isPermitted(
-        new WildcardPermission(PERMISSION_PREFIX + ":" + interceptedMethod.getName())));
+        Permissions.createUnscopedPermission(DOMAIN, interceptedMethod.getName())));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/aurora/blob/352e0ef5/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
index 781cf5a..33783c8 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
@@ -29,6 +29,7 @@ import org.apache.aurora.gen.ResponseCode;
 import org.apache.aurora.gen.TaskQuery;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.Query;
+import org.apache.aurora.scheduler.spi.Permissions.Domain;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.thrift.Responses;
 import org.apache.aurora.scheduler.thrift.aop.AnnotatedAuroraAdmin;
@@ -47,7 +48,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 
 public class ShiroAuthorizingParamInterceptorTest extends EasyMockTest {
-  private static final String PERMISSION_PREFIX = "testperm";
+  private static final Domain DOMAIN = Domain.THRIFT_AURORA_SCHEDULER_MANAGER;
 
   private ShiroAuthorizingParamInterceptor interceptor;
 
@@ -61,7 +62,7 @@ public class ShiroAuthorizingParamInterceptorTest extends EasyMockTest {
 
   @Before
   public void setUp() {
-    interceptor = new ShiroAuthorizingParamInterceptor(PERMISSION_PREFIX);
+    interceptor = new ShiroAuthorizingParamInterceptor(DOMAIN);
     subject = createMock(Subject.class);
     statsProvider = createMock(StatsProvider.class);
     thrift = createMock(AnnotatedAuroraAdmin.class);