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/10/22 01:14:43 UTC

[2/2] aurora git commit: Ignore all SessionKeys.

Ignore all SessionKeys.

This change ignores all SessionKeys. Since the scheduler was doing
"requireNonNull" on SessionKey before the client needs to continue
to set SessionKey to a non-null value for this release. In the next
release the parameter can be dropped completely.

Apologies for the large diff, thankfully it's mostly red. Coverage in
SchedulerThriftInterface is very high, most of the deleted tests deal
with branches related to authentication or authorization failures,
as these are now enforced in a different layer.

Testing Done:
./graldew -Pq clean build

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


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

Branch: refs/heads/master
Commit: 4eeec7a7513b7ab985613d7053361f51dd114eeb
Parents: 5609fc2
Author: Kevin Sweeney <ke...@apache.org>
Authored: Wed Oct 21 16:14:15 2015 -0700
Committer: Kevin Sweeney <ke...@apache.org>
Committed: Wed Oct 21 16:14:15 2015 -0700

----------------------------------------------------------------------
 NEWS                                            |   2 +
 .../apache/aurora/auth/CapabilityValidator.java |  60 ---
 .../apache/aurora/auth/SessionValidator.java    |  68 ---
 .../apache/aurora/auth/UnsecureAuthModule.java  | 103 -----
 .../aurora/auth/UnsecureSessionContext.java     |  48 --
 .../aurora/scheduler/app/MoreModules.java       |  30 --
 .../scheduler/http/JettyServerModule.java       |   4 +-
 .../http/api/GsonMessageBodyHandler.java        |  22 -
 .../aurora/scheduler/thrift/AuditMessages.java  |  19 +-
 .../thrift/SchedulerThriftInterface.java        | 283 +++---------
 .../aurora/scheduler/thrift/aop/AopModule.java  |  19 +-
 .../thrift/aop/LoggingInterceptor.java          |  28 --
 .../thrift/aop/UserCapabilityInterceptor.java   | 105 -----
 .../aurora/scheduler/thrift/auth/Requires.java  |  34 --
 .../scheduler/thrift/auth/ThriftAuthModule.java |  83 ----
 .../aurora/auth/UnsecureSessionContextTest.java |  65 ---
 .../aurora/scheduler/http/api/ApiBetaTest.java  |  10 +-
 .../scheduler/thrift/AuditMessagesTest.java     |  13 +-
 .../thrift/SchedulerThriftInterfaceTest.java    | 457 ++-----------------
 .../aurora/scheduler/thrift/ThriftIT.java       | 102 +----
 .../scheduler/thrift/aop/AopModuleTest.java     |  14 +-
 .../thrift/aop/LoggingInterceptorTest.java      |  24 +-
 22 files changed, 111 insertions(+), 1482 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/NEWS
----------------------------------------------------------------------
diff --git a/NEWS b/NEWS
index 7622f9d..40fe139 100644
--- a/NEWS
+++ b/NEWS
@@ -15,6 +15,8 @@
   for more details on Mesos runtime dependencies.
 - Resource quota is no longer consumed by production jobs with a dedicated constraint (AURORA-1457).
 - Thrift ConfigGroup.instanceIds field has been deprecated. Use ConfigGroup.instances instead.
+- Deprecated SessionValidator and CapabilityValidator interfaces have been removed. All
+  SessionKey-typed arguments are now nullable and ignored by the scheduler Thrift API.
 
 0.9.0
 -----

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/auth/CapabilityValidator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/auth/CapabilityValidator.java b/src/main/java/org/apache/aurora/auth/CapabilityValidator.java
deleted file mode 100644
index 198cdf3..0000000
--- a/src/main/java/org/apache/aurora/auth/CapabilityValidator.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * 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.auth;
-
-import org.apache.aurora.gen.SessionKey;
-
-/**
- * A session validator that supports user capability matching.
- * <p>
- * This supports asking whether a user has been granted a specific administration capability.
- */
-public interface CapabilityValidator extends SessionValidator {
-
-  enum Capability {
-    ROOT,
-    MACHINE_MAINTAINER,
-    PROVISIONER,
-    UPDATE_COORDINATOR,
-  }
-
-  /**
-   * Audit check to perform for a given action.
-   */
-  enum AuditCheck {
-    /**
-     * Default. No audit checks will be performed.
-     */
-    NONE,
-
-    /**
-     * A check will be performed to verify if a given action has
-     * all necessary data to generate a valid audit trail.
-     */
-    REQUIRED
-  }
-
-  /**
-   * Checks whether a session key is authenticated, and has the specified capability.
-   *
-   * @param sessionKey Key to validate.
-   * @param capability User capability to authenticate against.
-   * @param check Auditing data presence check required.
-   * @return  A {@link SessionContext} object that provides information about the validated session.
-   * @throws AuthFailedException If the key cannot be validated as the role or lacks
-   * the requested capability.
-   */
-  SessionContext checkAuthorized(SessionKey sessionKey, Capability capability, AuditCheck check)
-      throws AuthFailedException;
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/auth/SessionValidator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/auth/SessionValidator.java b/src/main/java/org/apache/aurora/auth/SessionValidator.java
deleted file mode 100644
index b688a0f..0000000
--- a/src/main/java/org/apache/aurora/auth/SessionValidator.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * 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.auth;
-
-import java.util.Set;
-
-import org.apache.aurora.gen.SessionKey;
-
-/**
- * Validator for RPC sessions with Aurora.
- */
-public interface SessionValidator {
-
-  /**
-   * Checks whether a session key is authenticated, and has permission to act as all the roles
-   * provided. Authentication is successful only if the SessionKey is successfully validated against
-   * all the roles.
-   *
-   * @param sessionKey Key to validate.
-   * @param targetRoles A set of roles to validate the key against.
-   * @return A {@link SessionContext} object that provides information about the validated session.
-   * @throws AuthFailedException If the key cannot be validated against a role.
-   */
-  SessionContext checkAuthenticated(SessionKey sessionKey, Set<String> targetRoles)
-      throws AuthFailedException;
-
-  /**
-   * Translates a {@link SessionKey} to a string. Primarily provides a way for the binary data
-   * within a {@link SessionKey} to be decoded and converted into a string.
-   *
-   * @param sessionKey The session key to translate.
-   * @return A string representation of the {@link SessionKey}.
-   */
-  String toString(SessionKey sessionKey);
-
-  /**
-   * Provides information about a session.
-   */
-  interface SessionContext {
-
-    /**
-     * Provides the identity for a validated session.
-     *
-     * @return A string that identifies the session.
-     */
-    String getIdentity();
-  }
-
-  /**
-   * Thrown when authentication is not successful.
-   */
-  class AuthFailedException extends Exception {
-    public AuthFailedException(String msg) {
-      super(msg);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/auth/UnsecureAuthModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/auth/UnsecureAuthModule.java b/src/main/java/org/apache/aurora/auth/UnsecureAuthModule.java
deleted file mode 100644
index 476dd9a..0000000
--- a/src/main/java/org/apache/aurora/auth/UnsecureAuthModule.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * 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.auth;
-
-import java.util.Set;
-import java.util.logging.Logger;
-
-import com.google.inject.AbstractModule;
-import com.google.inject.Inject;
-
-import org.apache.aurora.auth.SessionValidator.SessionContext;
-import org.apache.aurora.gen.SessionKey;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * An authentication module that uses an {@link UnsecureSessionValidator}. This behavior
- * can be overridden by binding a secure validator, querying an internal authentication system,
- * to {@link SessionValidator}.
- */
-public class UnsecureAuthModule extends AbstractModule {
-  private static final Logger LOG = Logger.getLogger(UnsecureAuthModule.class.getName());
-
-  @Override
-  protected void configure() {
-    LOG.info("Using default (UNSECURE!!!) authentication module.");
-    bind(SessionValidator.class).to(UnsecureSessionValidator.class);
-    bind(CapabilityValidator.class).to(UnsecureCapabilityValidator.class);
-    // NOTE: This binding is very important, as UnsecureSessionContext has an optional injection,
-    // so its provider must be created in the same injector as the one that *might* have its
-    // optional dependency. Omitting this binding will cause a Just-In-Time binding to be created
-    // in the parent injector, where it will not have access to the optional dependency in the
-    // child injector (so its optional dependency will never be used). This was the cause of
-    // https://issues.apache.org/jira/browse/AURORA-1352. This can be mitigated slightly by
-    // https://issues.apache.org/jira/browse/AURORA-1357
-    bind(SessionContext.class).to(UnsecureSessionContext.class);
-  }
-
-  static class UnsecureSessionValidator implements SessionValidator {
-    private final SessionContext sessionContext;
-
-    @Inject
-    UnsecureSessionValidator(SessionContext sessionContext) {
-      this.sessionContext = requireNonNull(sessionContext);
-    }
-
-    @Override
-    public SessionContext checkAuthenticated(SessionKey key, Set<String> targetRoles)
-        throws AuthFailedException {
-
-      return sessionContext;
-    }
-
-    @Override
-    public String toString(SessionKey sessionKey) {
-      return sessionKey.toString();
-    }
-  }
-
-  static class UnsecureCapabilityValidator implements CapabilityValidator {
-    private final SessionValidator sessionValidator;
-    private final SessionContext sessionContext;
-
-    @Inject
-    UnsecureCapabilityValidator(
-        SessionValidator sessionValidator,
-        SessionContext sessionContext) {
-
-      this.sessionValidator = requireNonNull(sessionValidator);
-      this.sessionContext = requireNonNull(sessionContext);
-    }
-
-    @Override
-    public SessionContext checkAuthorized(SessionKey key, Capability capability, AuditCheck check)
-        throws AuthFailedException {
-
-      return sessionContext;
-    }
-
-    @Override
-    public SessionContext checkAuthenticated(SessionKey key, Set<String> targetRoles)
-        throws AuthFailedException {
-
-      return sessionValidator.checkAuthenticated(key, targetRoles);
-    }
-
-    @Override
-    public String toString(SessionKey sessionKey) {
-      return sessionValidator.toString(sessionKey);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/auth/UnsecureSessionContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/auth/UnsecureSessionContext.java b/src/main/java/org/apache/aurora/auth/UnsecureSessionContext.java
deleted file mode 100644
index 47f81aa..0000000
--- a/src/main/java/org/apache/aurora/auth/UnsecureSessionContext.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * 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.auth;
-
-import java.util.Objects;
-import java.util.Optional;
-
-import javax.inject.Provider;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.inject.Inject;
-
-import org.apache.shiro.subject.Subject;
-
-/**
- * Uses context from Shiro for audit messages if available, otherwise defaults to a placeholder
- * indicating the audit record is unsecure.
- */
-class UnsecureSessionContext implements SessionValidator.SessionContext {
-  @VisibleForTesting
-  static final String UNSECURE = "UNSECURE";
-
-  private final Provider<Optional<Subject>> subjectProvider;
-
-  @Inject
-  UnsecureSessionContext(Provider<Optional<Subject>> subjectProvider) {
-    this.subjectProvider = Objects.requireNonNull(subjectProvider);
-  }
-
-  @Override
-  public String getIdentity() {
-    return subjectProvider.get()
-        .map(Subject::getPrincipals)
-        .map(principalCollection -> principalCollection.oneByType(String.class))
-        .orElse(UNSECURE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java b/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java
index b79ccc8..d5f9654 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java
@@ -15,15 +15,11 @@ package org.apache.aurora.scheduler.app;
 
 import com.google.inject.AbstractModule;
 import com.google.inject.Module;
-import com.google.inject.PrivateModule;
-
-import static java.util.Objects.requireNonNull;
 
 /**
  * A utility class for managing guice modules.
  */
 public final class MoreModules {
-
   private MoreModules() {
     // Utility class
   }
@@ -46,32 +42,6 @@ public final class MoreModules {
     }
   }
 
-  /**
-   * Defensively wrap a module in a PrivateModule that only exposes requested keys to ensure that
-   * we don't depend on surprise extra bindings across different implementations.
-   *
-   * @param module Module to wrap.
-   * @param exposedClasses Keys to expose.
-   * @return A private module that exposes the requested keys.
-   */
-  public static Module wrapInPrivateModule(
-      final Module module,
-      final Iterable<Class<?>> exposedClasses) {
-
-    requireNonNull(module);
-    requireNonNull(exposedClasses);
-
-    return new PrivateModule() {
-      @Override
-      protected void configure() {
-        install(module);
-        for (Class<?> klass : exposedClasses) {
-          expose(klass);
-        }
-      }
-    };
-  }
-
   static Module getModule(Class<? extends Module> moduleClass) {
     return instantiateModule(moduleClass);
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/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 ee024a3..c78315d 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
@@ -70,7 +70,6 @@ import org.apache.aurora.scheduler.SchedulerServicesModule;
 import org.apache.aurora.scheduler.http.api.ApiModule;
 import org.apache.aurora.scheduler.http.api.security.HttpSecurityModule;
 import org.apache.aurora.scheduler.thrift.ThriftModule;
-import org.apache.aurora.scheduler.thrift.auth.ThriftAuthModule;
 import org.eclipse.jetty.rewrite.handler.RewriteHandler;
 import org.eclipse.jetty.rewrite.handler.RewriteRegexRule;
 import org.eclipse.jetty.server.Connector;
@@ -192,8 +191,7 @@ public class JettyServerModule extends AbstractModule {
               new ApiModule(),
               new H2ConsoleModule(),
               new HttpSecurityModule(),
-              new ThriftModule(),
-              new ThriftAuthModule()));
+              new ThriftModule()));
     }
   };
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/http/api/GsonMessageBodyHandler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/GsonMessageBodyHandler.java b/src/main/java/org/apache/aurora/scheduler/http/api/GsonMessageBodyHandler.java
index 2ec7da6..41f48b9 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/GsonMessageBodyHandler.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/GsonMessageBodyHandler.java
@@ -21,7 +21,6 @@ import java.io.OutputStreamWriter;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Field;
 import java.lang.reflect.Type;
-import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -40,7 +39,6 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
-import com.google.common.io.BaseEncoding;
 import com.google.gson.ExclusionStrategy;
 import com.google.gson.FieldAttributes;
 import com.google.gson.Gson;
@@ -52,9 +50,6 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
-import com.google.gson.TypeAdapter;
-import com.google.gson.stream.JsonReader;
-import com.google.gson.stream.JsonWriter;
 
 import org.apache.thrift.TFieldIdEnum;
 import org.apache.thrift.TUnion;
@@ -233,22 +228,5 @@ public class GsonMessageBodyHandler
           }
         }
       })
-      .registerTypeAdapter(ByteBuffer.class, new TypeAdapter<ByteBuffer>() {
-        @Override
-        public void write(JsonWriter out, ByteBuffer value) throws IOException {
-          out.value(BaseEncoding.base64().encode(value.array()));
-        }
-
-        @Override
-        public ByteBuffer read(JsonReader in) throws IOException {
-          try {
-            return ByteBuffer.wrap(BaseEncoding.base64().decode(in.nextString()));
-          } catch (UnsupportedOperationException e) {
-            throw new JsonParseException("Byte array element must be a JSON string.", e);
-          } catch (IllegalArgumentException e) {
-            throw new JsonParseException("Unable to parse base64-encoded string.", e);
-          }
-        }
-      })
       .create();
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java b/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java
index 3a29a62..a7aeced 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java
@@ -29,6 +29,9 @@ import org.apache.shiro.subject.Subject;
  */
 @VisibleForTesting
 class AuditMessages {
+  @VisibleForTesting
+  static final String DEFAULT_USER = "UNSECURE";
+
   private final Provider<Optional<Subject>> subjectProvider;
 
   @Inject
@@ -36,22 +39,22 @@ class AuditMessages {
     this.subjectProvider = Objects.requireNonNull(subjectProvider);
   }
 
-  private String getShiroUserNameOr(String defaultUser) {
+  String getRemoteUserName() {
     return subjectProvider.get()
         .map(Subject::getPrincipal)
         .map(Object::toString)
-        .orElse(defaultUser);
+        .orElse(DEFAULT_USER);
   }
 
-  com.google.common.base.Optional<String> transitionedBy(String user) {
-    return com.google.common.base.Optional.of("Transition forced by " + getShiroUserNameOr(user));
+  com.google.common.base.Optional<String> transitionedBy() {
+    return com.google.common.base.Optional.of("Transition forced by " + getRemoteUserName());
   }
 
-  com.google.common.base.Optional<String> killedBy(String user) {
-    return com.google.common.base.Optional.of("Killed by " + getShiroUserNameOr(user));
+  com.google.common.base.Optional<String> killedByRemoteUser() {
+    return com.google.common.base.Optional.of("Killed by " + getRemoteUserName());
   }
 
-  com.google.common.base.Optional<String> restartedBy(String user) {
-    return com.google.common.base.Optional.of("Restarted by " + getShiroUserNameOr(user));
+  com.google.common.base.Optional<String> restartedByRemoteUser() {
+    return com.google.common.base.Optional.of("Restarted by " + getRemoteUserName());
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
index 304437e..ccabf55 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -22,8 +22,6 @@ import javax.annotation.Nullable;
 import javax.inject.Inject;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.Functions;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.FluentIterable;
@@ -36,9 +34,6 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
 import com.google.common.collect.Range;
 
-import org.apache.aurora.auth.CapabilityValidator;
-import org.apache.aurora.auth.CapabilityValidator.AuditCheck;
-import org.apache.aurora.auth.SessionValidator.AuthFailedException;
 import org.apache.aurora.common.args.Arg;
 import org.apache.aurora.common.args.CmdLine;
 import org.apache.aurora.common.args.constraints.Positive;
@@ -62,7 +57,6 @@ import org.apache.aurora.gen.JobUpdateSummary;
 import org.apache.aurora.gen.ListBackupsResult;
 import org.apache.aurora.gen.Lock;
 import org.apache.aurora.gen.LockKey;
-import org.apache.aurora.gen.LockKey._Fields;
 import org.apache.aurora.gen.LockValidation;
 import org.apache.aurora.gen.MaintenanceStatusResult;
 import org.apache.aurora.gen.PulseJobUpdateResult;
@@ -98,7 +92,6 @@ import org.apache.aurora.scheduler.state.StateChangeResult;
 import org.apache.aurora.scheduler.state.StateManager;
 import org.apache.aurora.scheduler.state.UUIDGenerator;
 import org.apache.aurora.scheduler.storage.CronJobStore;
-import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork;
 import org.apache.aurora.scheduler.storage.Storage.NonVolatileStorage;
@@ -124,7 +117,6 @@ import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.apache.aurora.scheduler.thrift.aop.AnnotatedAuroraAdmin;
 import org.apache.aurora.scheduler.thrift.auth.DecoratedThrift;
-import org.apache.aurora.scheduler.thrift.auth.Requires;
 import org.apache.aurora.scheduler.updater.JobDiff;
 import org.apache.aurora.scheduler.updater.JobUpdateController;
 import org.apache.aurora.scheduler.updater.JobUpdateController.AuditData;
@@ -135,13 +127,7 @@ import static java.util.Objects.requireNonNull;
 
 import static com.google.common.base.CharMatcher.WHITESPACE;
 
-import static org.apache.aurora.auth.CapabilityValidator.Capability.MACHINE_MAINTAINER;
-import static org.apache.aurora.auth.CapabilityValidator.Capability.PROVISIONER;
-import static org.apache.aurora.auth.CapabilityValidator.Capability.ROOT;
-import static org.apache.aurora.auth.CapabilityValidator.Capability.UPDATE_COORDINATOR;
-import static org.apache.aurora.auth.SessionValidator.SessionContext;
 import static org.apache.aurora.common.base.MorePreconditions.checkNotBlank;
-import static org.apache.aurora.gen.ResponseCode.AUTH_FAILED;
 import static org.apache.aurora.gen.ResponseCode.INVALID_REQUEST;
 import static org.apache.aurora.gen.ResponseCode.LOCK_ERROR;
 import static org.apache.aurora.gen.ResponseCode.OK;
@@ -187,13 +173,8 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
 
   private static final Logger LOG = Logger.getLogger(SchedulerThriftInterface.class.getName());
 
-  private static final Function<IScheduledTask, String> GET_ROLE = Functions.compose(
-      task -> task.getJob().getRole(),
-      Tasks::getConfig);
-
   private final NonVolatileStorage storage;
   private final LockManager lockManager;
-  private final CapabilityValidator sessionValidator;
   private final StorageBackup backup;
   private final Recovery recovery;
   private final MaintenanceController maintenance;
@@ -210,7 +191,6 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   SchedulerThriftInterface(
       NonVolatileStorage storage,
       LockManager lockManager,
-      CapabilityValidator sessionValidator,
       StorageBackup backup,
       Recovery recovery,
       CronJobManager cronJobManager,
@@ -225,7 +205,6 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
 
     this.storage = requireNonNull(storage);
     this.lockManager = requireNonNull(lockManager);
-    this.sessionValidator = requireNonNull(sessionValidator);
     this.backup = requireNonNull(backup);
     this.recovery = requireNonNull(recovery);
     this.maintenance = requireNonNull(maintenance);
@@ -243,18 +222,11 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   public Response createJob(
       JobConfiguration mutableJob,
       @Nullable final Lock mutableLock,
-      SessionKey session) {
-
-    requireNonNull(session);
+      @Nullable SessionKey ignored) {
 
     final SanitizedConfiguration sanitized;
     try {
-      sessionValidator.checkAuthenticated(
-          session,
-          ImmutableSet.of(mutableJob.getKey().getRole()));
       sanitized = SanitizedConfiguration.fromUnsanitized(IJobConfiguration.build(mutableJob));
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
     } catch (TaskDescriptionException e) {
       return error(INVALID_REQUEST, e);
     }
@@ -316,19 +288,14 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   private Response createOrUpdateCronTemplate(
       JobConfiguration mutableJob,
       @Nullable final Lock mutableLock,
-      SessionKey session,
       final boolean updateOnly) {
 
     IJobConfiguration job = IJobConfiguration.build(mutableJob);
     final IJobKey jobKey = JobKeys.assertValid(job.getKey());
-    requireNonNull(session);
 
     final SanitizedConfiguration sanitized;
     try {
-      sessionValidator.checkAuthenticated(session, ImmutableSet.of(jobKey.getRole()));
       sanitized = SanitizedConfiguration.fromUnsanitized(job);
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
     } catch (TaskDescriptionException e) {
       return error(INVALID_REQUEST, e);
     }
@@ -376,29 +343,27 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   public Response scheduleCronJob(
       JobConfiguration mutableJob,
       @Nullable Lock mutableLock,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
-    return createOrUpdateCronTemplate(mutableJob, mutableLock, session, false);
+    return createOrUpdateCronTemplate(mutableJob, mutableLock, false);
   }
 
   @Override
   public Response replaceCronTemplate(
       JobConfiguration mutableJob,
       @Nullable Lock mutableLock,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
-    return createOrUpdateCronTemplate(mutableJob, mutableLock, session, true);
+    return createOrUpdateCronTemplate(mutableJob, mutableLock, true);
   }
 
   @Override
   public Response descheduleCronJob(
       JobKey mutableJobKey,
       @Nullable Lock mutableLock,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     try {
-      sessionValidator.checkAuthenticated(session, ImmutableSet.of(mutableJobKey.getRole()));
-
       IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJobKey));
       lockManager.validateIfLocked(
           ILockKey.build(LockKey.job(jobKey.newBuilder())),
@@ -408,8 +373,6 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
         return invalidRequest(notScheduledCronMessage(jobKey));
       }
       return ok();
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
     } catch (LockException e) {
       return error(LOCK_ERROR, e);
     }
@@ -421,17 +384,10 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response startCronJob(JobKey mutableJobKey, SessionKey session) {
-    requireNonNull(session);
+  public Response startCronJob(JobKey mutableJobKey, @Nullable SessionKey ignored) {
     IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJobKey));
 
     try {
-      sessionValidator.checkAuthenticated(session, ImmutableSet.of(jobKey.getRole()));
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
-    }
-
-    try {
       cronJobManager.startJobNow(jobKey);
       return ok();
     } catch (CronException e) {
@@ -503,52 +459,13 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
     return query.get().isSetStatuses() ? query : query.byStatus(ACTIVE_STATES);
   }
 
-  private SessionContext authenticateNonAdminKillingTasks(
-      Query.Builder taskQuery,
-      SessionKey session) throws AuthFailedException {
-
-    // Authenticate the session against any affected roles, always including the role for a
-    // role-scoped query.
-    ImmutableSet.Builder<String> targetRoles = ImmutableSet.builder();
-    Set<IJobKey> keys = JobKeys.from(taskQuery).or(ImmutableSet.of());
-    targetRoles.addAll(FluentIterable.from(keys).transform(IJobKey::getRole));
-
-    if (taskQuery.get().isSetRole()) {
-      targetRoles.add(taskQuery.get().getRole());
-    }
-
-    if (taskQuery.get().isSetTaskIds()) {
-      // Note: this operation is weakly-consistent with respect to the transaction that performs
-      // the kill.  This means the task could exit between authentication and kill.  Since the user
-      // asked to kill a task ID (rather than an instance ID, which would outlast a task), this is
-      // considered acceptable.
-      targetRoles.addAll(
-          FluentIterable.from(Storage.Util.fetchTasks(storage, taskQuery)).transform(GET_ROLE));
-    }
-
-    Set<String> explicitRoles = targetRoles.build();
-    // Disallow non-admin users from killing tasks on behalf of arbitrary roles.  Since query fields
-    // are AND-ed together, the presence of a role scope on the query is sufficient for a non-admin
-    // user to perform the operation.
-    if (explicitRoles.isEmpty()) {
-      throw new AuthFailedException("Only an administrator may kill an arbitrary user's tasks.");
-    } else {
-      return sessionValidator.checkAuthenticated(session, explicitRoles);
-    }
-  }
-
-  private Optional<SessionContext> isAdmin(SessionKey session) {
-    try {
-      return Optional.of(sessionValidator.checkAuthorized(session, ROOT, AuditCheck.REQUIRED));
-    } catch (AuthFailedException e) {
-      return Optional.absent();
-    }
-  }
-
   @Override
-  public Response killTasks(TaskQuery mutableQuery, final Lock mutableLock, SessionKey session) {
+  public Response killTasks(
+      TaskQuery mutableQuery,
+      final Lock mutableLock,
+      @Nullable SessionKey ignored) {
+
     requireNonNull(mutableQuery);
-    requireNonNull(session);
 
     if (mutableQuery.getJobName() != null && WHITESPACE.matchesAllOf(mutableQuery.getJobName())) {
       return invalidRequest(String.format("Invalid job name: '%s'", mutableQuery.getJobName()));
@@ -559,19 +476,6 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
         !query.get().isSetOwner(),
         "The owner field in a query should have been unset by Query.Builder.");
 
-    Optional<SessionContext> maybeAdminContext = isAdmin(session);
-    final SessionContext context;
-    if (maybeAdminContext.isPresent()) {
-      LOG.info("Granting kill query to admin user: " + query);
-      context = maybeAdminContext.get();
-    } else {
-      try {
-        context = authenticateNonAdminKillingTasks(query, session);
-      } catch (AuthFailedException e) {
-        return error(AUTH_FAILED, e);
-      }
-    }
-
     return storage.write(new MutateWork.Quiet<Response>() {
       @Override
       public Response apply(MutableStoreProvider storeProvider) {
@@ -593,7 +497,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
               taskId,
               Optional.absent(),
               ScheduleStatus.KILLING,
-              auditMessages.killedBy(context.getIdentity()));
+              auditMessages.killedByRemoteUser());
         }
 
         return tasksKilled
@@ -608,18 +512,10 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
       JobKey mutableJobKey,
       final Set<Integer> shardIds,
       @Nullable final Lock mutableLock,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     final IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJobKey));
     checkNotBlank(shardIds);
-    requireNonNull(session);
-
-    final SessionContext context;
-    try {
-      context = sessionValidator.checkAuthenticated(session, ImmutableSet.of(jobKey.getRole()));
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
-    }
 
     return storage.write(new MutateWork.Quiet<Response>() {
       @Override
@@ -649,7 +545,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
                   taskId,
                   Optional.absent(),
                   ScheduleStatus.RESTARTING,
-                  auditMessages.restartedBy(context.getIdentity()));
+                  auditMessages.restartedByRemoteUser());
             }
           }
         });
@@ -663,16 +559,14 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
     return readOnlyScheduler.getQuota(ownerRole);
   }
 
-  @Requires(whitelist = PROVISIONER)
   @Override
   public Response setQuota(
       final String ownerRole,
       final ResourceAggregate resourceAggregate,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     checkNotBlank(ownerRole);
     requireNonNull(resourceAggregate);
-    requireNonNull(session);
 
     try {
       storage.write(new MutateWork.NoResult<QuotaException>() {
@@ -690,31 +584,27 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
     }
   }
 
-  @Requires(whitelist = MACHINE_MAINTAINER)
   @Override
-  public Response startMaintenance(Hosts hosts, SessionKey session) {
+  public Response startMaintenance(Hosts hosts, @Nullable SessionKey ignored) {
     return ok(Result.startMaintenanceResult(
         new StartMaintenanceResult()
             .setStatuses(maintenance.startMaintenance(hosts.getHostNames()))));
   }
 
-  @Requires(whitelist = MACHINE_MAINTAINER)
   @Override
-  public Response drainHosts(Hosts hosts, SessionKey session) {
+  public Response drainHosts(Hosts hosts, @Nullable SessionKey ignored) {
     return ok(Result.drainHostsResult(
         new DrainHostsResult().setStatuses(maintenance.drain(hosts.getHostNames()))));
   }
 
-  @Requires(whitelist = MACHINE_MAINTAINER)
   @Override
-  public Response maintenanceStatus(Hosts hosts, SessionKey session) {
+  public Response maintenanceStatus(Hosts hosts, @Nullable SessionKey ignored) {
     return ok(Result.maintenanceStatusResult(
         new MaintenanceStatusResult().setStatuses(maintenance.getStatus(hosts.getHostNames()))));
   }
 
-  @Requires(whitelist = MACHINE_MAINTAINER)
   @Override
-  public Response endMaintenance(Hosts hosts, SessionKey session) {
+  public Response endMaintenance(Hosts hosts, @Nullable SessionKey ignored) {
     return ok(Result.endMaintenanceResult(
         new EndMaintenanceResult()
             .setStatuses(maintenance.endMaintenance(hosts.getHostNames()))));
@@ -724,19 +614,10 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   public Response forceTaskState(
       final String taskId,
       final ScheduleStatus status,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     checkNotBlank(taskId);
     requireNonNull(status);
-    requireNonNull(session);
-
-    final SessionContext context;
-    try {
-      // TODO(maxim): Remove this after AOP-style session validation passes in a SessionContext.
-      context = sessionValidator.checkAuthorized(session, ROOT, AuditCheck.REQUIRED);
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
-    }
 
     storage.write(new MutateWork.NoResult.Quiet() {
       @Override
@@ -746,7 +627,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
             taskId,
             Optional.absent(),
             status,
-            auditMessages.transitionedBy(context.getIdentity()));
+            auditMessages.transitionedBy());
       }
     });
 
@@ -754,55 +635,58 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response performBackup(SessionKey session) {
+  public Response performBackup(@Nullable SessionKey ignored) {
     backup.backupNow();
     return ok();
   }
 
   @Override
-  public Response listBackups(SessionKey session) {
+  public Response listBackups(@Nullable SessionKey ignored) {
     return ok(Result.listBackupsResult(new ListBackupsResult()
         .setBackups(recovery.listBackups())));
   }
 
   @Override
-  public Response stageRecovery(String backupId, SessionKey session) {
+  public Response stageRecovery(String backupId, @Nullable SessionKey ignored) {
     recovery.stage(backupId);
     return ok();
   }
 
   @Override
-  public Response queryRecovery(TaskQuery query, SessionKey session) {
+  public Response queryRecovery(TaskQuery query, @Nullable SessionKey ignored) {
     return ok(Result.queryRecoveryResult(new QueryRecoveryResult()
         .setTasks(IScheduledTask.toBuildersSet(recovery.query(Query.arbitrary(query))))));
   }
 
   @Override
-  public Response deleteRecoveryTasks(TaskQuery query, SessionKey session) {
+  public Response deleteRecoveryTasks(TaskQuery query, @Nullable SessionKey ignored) {
     recovery.deleteTasks(Query.arbitrary(query));
     return ok();
   }
 
   @Override
-  public Response commitRecovery(SessionKey session) {
+  public Response commitRecovery(@Nullable SessionKey ignored) {
     recovery.commit();
     return ok();
   }
 
   @Override
-  public Response unloadRecovery(SessionKey session) {
+  public Response unloadRecovery(@Nullable SessionKey ignored) {
     recovery.unload();
     return ok();
   }
 
   @Override
-  public Response snapshot(SessionKey session) {
+  public Response snapshot(@Nullable SessionKey ignored) {
     storage.snapshot();
     return ok();
   }
 
   @Override
-  public Response rewriteConfigs(final RewriteConfigsRequest request, SessionKey session) {
+  public Response rewriteConfigs(
+      final RewriteConfigsRequest request,
+      @Nullable SessionKey ignored) {
+
     if (request.getRewriteCommandsSize() == 0) {
       return addMessage(Responses.empty(), INVALID_REQUEST, "No rewrite commands provided.");
     }
@@ -922,20 +806,16 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   public Response addInstances(
       final AddInstancesConfig config,
       @Nullable final Lock mutableLock,
-      final SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     requireNonNull(config);
-    requireNonNull(session);
     checkNotBlank(config.getInstanceIds());
     final IJobKey jobKey = JobKeys.assertValid(IJobKey.build(config.getKey()));
 
     final ITaskConfig task;
     try {
-      sessionValidator.checkAuthenticated(session, ImmutableSet.of(jobKey.getRole()));
       task = ConfigurationManager.validateAndPopulate(
           ITaskConfig.build(config.getTaskConfig()));
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
     } catch (TaskDescriptionException e) {
       return error(INVALID_REQUEST, e);
     }
@@ -985,57 +865,38 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
     return storeProvider.getCronJobStore().fetchJob(jobKey);
   }
 
-  private String getRoleFromLockKey(ILockKey lockKey) {
-    if (lockKey.getSetField() == _Fields.JOB) {
-      JobKeys.assertValid(lockKey.getJob());
-      return lockKey.getJob().getRole();
-    } else {
-      throw new IllegalArgumentException("Unhandled LockKey: " + lockKey.getSetField());
-    }
-  }
-
   @Override
-  public Response acquireLock(LockKey mutableLockKey, SessionKey session) {
+  public Response acquireLock(LockKey mutableLockKey, @Nullable SessionKey ignored) {
     requireNonNull(mutableLockKey);
-    requireNonNull(session);
 
     ILockKey lockKey = ILockKey.build(mutableLockKey);
 
     try {
-      SessionContext context = sessionValidator.checkAuthenticated(
-          session,
-          ImmutableSet.of(getRoleFromLockKey(lockKey)));
-
-      ILock lock = lockManager.acquireLock(lockKey, context.getIdentity());
+      ILock lock = lockManager.acquireLock(lockKey, auditMessages.getRemoteUserName());
       return ok(Result.acquireLockResult(
           new AcquireLockResult().setLock(lock.newBuilder())));
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
     } catch (LockException e) {
       return error(LOCK_ERROR, e);
     }
   }
 
   @Override
-  public Response releaseLock(Lock mutableLock, LockValidation validation, SessionKey session) {
+  public Response releaseLock(
+      Lock mutableLock,
+      LockValidation validation,
+      @Nullable SessionKey ignored) {
+
     requireNonNull(mutableLock);
     requireNonNull(validation);
-    requireNonNull(session);
 
     ILock lock = ILock.build(mutableLock);
 
     try {
-      sessionValidator.checkAuthenticated(
-          session,
-          ImmutableSet.of(getRoleFromLockKey(lock.getKey())));
-
       if (validation == LockValidation.CHECKED) {
         lockManager.validateIfLocked(lock.getKey(), java.util.Optional.of(lock));
       }
       lockManager.releaseLock(lock);
       return ok();
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
     } catch (LockException e) {
       return error(LOCK_ERROR, e);
     }
@@ -1099,10 +960,9 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   public Response startJobUpdate(
       JobUpdateRequest mutableRequest,
       @Nullable final String message,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     requireNonNull(mutableRequest);
-    requireNonNull(session);
 
     // TODO(maxim): Switch to key field instead when AURORA-749 is fixed.
     final IJobKey job = JobKeys.assertValid(IJobKey.build(new JobKey()
@@ -1140,16 +1000,12 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
       return invalidRequest(INVALID_PULSE_TIMEOUT);
     }
 
-    final SessionContext context;
     final IJobUpdateRequest request;
     try {
-      context = sessionValidator.checkAuthenticated(session, ImmutableSet.of(job.getRole()));
       request = IJobUpdateRequest.build(new JobUpdateRequest(mutableRequest).setTaskConfig(
           ConfigurationManager.validateAndPopulate(
               ITaskConfig.build(mutableRequest.getTaskConfig())).newBuilder()));
 
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
     } catch (TaskDescriptionException e) {
       return error(INVALID_REQUEST, e);
     }
@@ -1195,10 +1051,11 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
                   .setInstances(IRange.toBuildersSet(convertRanges(toRanges(replacements)))));
         }
 
+        String remoteUserName = auditMessages.getRemoteUserName();
         IJobUpdate update = IJobUpdate.build(new JobUpdate()
             .setSummary(new JobUpdateSummary()
                 .setKey(new JobUpdateKey(job.newBuilder(), updateId))
-                .setUser(context.getIdentity()))
+                .setUser(remoteUserName))
             .setInstructions(instructions));
         try {
           validateTaskLimits(
@@ -1208,7 +1065,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
 
           jobUpdateController.start(
               update,
-              new AuditData(context.getIdentity(), Optional.fromNullable(message)));
+              new AuditData(remoteUserName, Optional.fromNullable(message)));
           return ok(Result.startJobUpdateResult(
               new StartJobUpdateResult(update.getSummary().getKey().newBuilder())));
         } catch (UpdateStateException | TaskValidationException e) {
@@ -1220,18 +1077,11 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
 
   private Response changeJobUpdateState(
       JobUpdateKey mutableKey,
-      SessionKey session,
       final JobUpdateStateChange change,
       final Optional<String> message) {
 
     final IJobUpdateKey key = IJobUpdateKey.build(mutableKey);
     JobKeys.assertValid(key.getJob());
-    final SessionContext context;
-    try {
-      context = authorizeJobUpdateAction(key, session);
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
-    }
     return storage.write(new MutateWork.Quiet<Response>() {
       @Override
       public Response apply(MutableStoreProvider storeProvider) {
@@ -1239,7 +1089,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
           change.modifyUpdate(
               jobUpdateController,
               key,
-              new AuditData(context.getIdentity(), message));
+              new AuditData(auditMessages.getRemoteUserName(), message));
           return ok();
         } catch (UpdateStateException e) {
           return error(INVALID_REQUEST, e);
@@ -1257,11 +1107,10 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   public Response pauseJobUpdate(
       JobUpdateKey mutableKey,
       @Nullable String message,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     return changeJobUpdateState(
         mutableKey,
-        session,
         JobUpdateController::pause,
         Optional.fromNullable(message));
   }
@@ -1270,11 +1119,10 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   public Response resumeJobUpdate(
       JobUpdateKey mutableKey,
       @Nullable String message,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     return changeJobUpdateState(
         mutableKey,
-        session,
         JobUpdateController::resume,
         Optional.fromNullable(message));
   }
@@ -1283,24 +1131,20 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   public Response abortJobUpdate(
       JobUpdateKey mutableKey,
       @Nullable String message,
-      SessionKey session) {
+      @Nullable SessionKey ignored) {
 
     return changeJobUpdateState(
         mutableKey,
-        session,
         JobUpdateController::abort,
         Optional.fromNullable(message));
   }
 
   @Override
-  public Response pulseJobUpdate(JobUpdateKey mutableUpdateKey, final SessionKey session) {
+  public Response pulseJobUpdate(JobUpdateKey mutableUpdateKey, @Nullable SessionKey ignored) {
     IJobUpdateKey updateKey = validateJobUpdateKey(mutableUpdateKey);
     try {
-      authorizeJobUpdateAction(updateKey, session);
       JobUpdatePulseStatus result = jobUpdateController.pulse(updateKey);
       return ok(Result.pulseJobUpdateResult(new PulseJobUpdateResult(result)));
-    } catch (AuthFailedException e) {
-      return error(AUTH_FAILED, e);
     } catch (UpdateStateException e) {
       return error(INVALID_REQUEST, e);
     }
@@ -1316,31 +1160,6 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
     return readOnlyScheduler.getJobUpdateDetails(key);
   }
 
-  private Optional<SessionContext> isUpdateCoordinator(SessionKey session) {
-    try {
-      return Optional.of(
-          sessionValidator.checkAuthorized(session, UPDATE_COORDINATOR, AuditCheck.NONE));
-    } catch (AuthFailedException e) {
-      return Optional.absent();
-    }
-  }
-
-  private SessionContext authorizeJobUpdateAction(IJobUpdateKey key, SessionKey session)
-      throws AuthFailedException {
-
-    Optional<SessionContext> maybeCoordinatorContext = isUpdateCoordinator(session);
-    SessionContext context;
-    if (maybeCoordinatorContext.isPresent()) {
-      context = maybeCoordinatorContext.get();
-    } else {
-      context = sessionValidator.checkAuthenticated(
-          session,
-          ImmutableSet.of(key.getJob().getRole()));
-    }
-
-    return context;
-  }
-
   private static IJobUpdateKey validateJobUpdateKey(JobUpdateKey mutableKey) {
     IJobUpdateKey key = IJobUpdateKey.build(mutableKey);
     JobKeys.assertValid(key.getJob());

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/thrift/aop/AopModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/aop/AopModule.java b/src/main/java/org/apache/aurora/scheduler/thrift/aop/AopModule.java
index e951f8e..79b677d 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/aop/AopModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/aop/AopModule.java
@@ -37,11 +37,8 @@ import com.google.inject.matcher.Matcher;
 import com.google.inject.matcher.Matchers;
 
 import org.aopalliance.intercept.MethodInterceptor;
-import org.apache.aurora.GuiceUtils;
-import org.apache.aurora.auth.CapabilityValidator;
 import org.apache.aurora.common.args.Arg;
 import org.apache.aurora.common.args.CmdLine;
-import org.apache.aurora.gen.AuroraAdmin;
 import org.apache.aurora.gen.AuroraSchedulerManager;
 import org.apache.aurora.gen.Response;
 import org.apache.aurora.scheduler.thrift.auth.DecoratedThrift;
@@ -84,29 +81,15 @@ public class AopModule extends AbstractModule {
 
   @Override
   protected void configure() {
-    requireBinding(CapabilityValidator.class);
-
     // Layer ordering:
-    // APIVersion -> Log -> CapabilityValidator -> FeatureToggle -> StatsExporter ->
-    // SchedulerThriftInterface
+    // APIVersion -> Log -> FeatureToggle -> StatsExporter -> SchedulerThriftInterface
 
     // It's important for this interceptor to be registered first to ensure it's at the 'top' of
     // the stack and the standard message is always applied.
     bindThriftDecorator(new ServerInfoInterceptor());
 
-    // TODO(Sathya): Consider using provider pattern for constructing interceptors to facilitate
-    // unit testing without the creation of Guice injectors.
     bindThriftDecorator(new LoggingInterceptor());
 
-    // Note: it's important that the capability interceptor is only applied to AuroraAdmin.Iface
-    // methods, and does not pick up methods on AuroraSchedulerManager.Iface.
-    MethodInterceptor authInterceptor = new UserCapabilityInterceptor();
-    requestInjection(authInterceptor);
-    bindInterceptor(
-        THRIFT_IFACE_MATCHER,
-        GuiceUtils.interfaceMatcher(AuroraAdmin.Iface.class, true),
-        authInterceptor);
-
     install(new PrivateModule() {
       @Override
       protected void configure() {

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/thrift/aop/LoggingInterceptor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/aop/LoggingInterceptor.java b/src/main/java/org/apache/aurora/scheduler/thrift/aop/LoggingInterceptor.java
index d110b21..a7b9b07 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/aop/LoggingInterceptor.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/aop/LoggingInterceptor.java
@@ -18,9 +18,6 @@ import java.util.Map;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import javax.inject.Inject;
-
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
@@ -28,37 +25,20 @@ import com.google.common.collect.Lists;
 
 import org.aopalliance.intercept.MethodInterceptor;
 import org.aopalliance.intercept.MethodInvocation;
-import org.apache.aurora.auth.CapabilityValidator;
 import org.apache.aurora.gen.ExecutorConfig;
 import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.ResponseCode;
-import org.apache.aurora.gen.SessionKey;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.thrift.Responses;
 import org.apache.shiro.ShiroException;
 
-import static java.util.Objects.requireNonNull;
-
 /**
  * A method interceptor that logs all invocations as well as any unchecked exceptions thrown from
  * the underlying call.
  */
 class LoggingInterceptor implements MethodInterceptor {
-
   private static final Logger LOG = Logger.getLogger(LoggingInterceptor.class.getName());
 
-  @Inject
-  private CapabilityValidator validator;
-
-  LoggingInterceptor() {
-    // Guice constructor.
-  }
-
-  @VisibleForTesting
-  LoggingInterceptor(CapabilityValidator validator) {
-    this.validator = requireNonNull(validator);
-  }
-
   private final Map<Class<?>, Function<Object, String>> printFunctions =
       ImmutableMap.of(
           JobConfiguration.class,
@@ -72,14 +52,6 @@ class LoggingInterceptor implements MethodInterceptor {
               }
               return configuration.toString();
             }
-          },
-          SessionKey.class,
-          new Function<Object, String>() {
-            @Override
-            public String apply(Object input) {
-              SessionKey key = (SessionKey) input;
-              return validator.toString(key);
-            }
           }
       );
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/thrift/aop/UserCapabilityInterceptor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/aop/UserCapabilityInterceptor.java b/src/main/java/org/apache/aurora/scheduler/thrift/aop/UserCapabilityInterceptor.java
deleted file mode 100644
index dd8d272..0000000
--- a/src/main/java/org/apache/aurora/scheduler/thrift/aop/UserCapabilityInterceptor.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/**
- * 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.thrift.aop;
-
-import java.lang.reflect.Method;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-import java.util.logging.Logger;
-
-import javax.inject.Inject;
-
-import com.google.common.base.Function;
-import com.google.common.base.Optional;
-import com.google.common.base.Predicates;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-
-import org.aopalliance.intercept.MethodInterceptor;
-import org.aopalliance.intercept.MethodInvocation;
-import org.apache.aurora.auth.CapabilityValidator;
-import org.apache.aurora.auth.CapabilityValidator.AuditCheck;
-import org.apache.aurora.auth.CapabilityValidator.Capability;
-import org.apache.aurora.auth.SessionValidator.AuthFailedException;
-import org.apache.aurora.gen.ResponseCode;
-import org.apache.aurora.gen.SessionKey;
-import org.apache.aurora.scheduler.thrift.Responses;
-import org.apache.aurora.scheduler.thrift.auth.Requires;
-
-/**
- * A method interceptor that will authenticate users identified by a {@link SessionKey} argument
- * to invoked methods.
- * <p>
- * Intercepted methods will require {@link Capability#ROOT}, but additional capabilities
- * may be specified by annotating methods with {@link Requires} and supplying a whitelist.
- */
-class UserCapabilityInterceptor implements MethodInterceptor {
-  private static final Logger LOG = Logger.getLogger(UserCapabilityInterceptor.class.getName());
-
-  @Inject private CapabilityValidator capabilityValidator;
-
-  private static final Function<Object, SessionKey> CAST = new Function<Object, SessionKey>() {
-    @Override
-    public SessionKey apply(Object o) {
-      return (SessionKey) o;
-    }
-  };
-
-  @Override
-  public Object invoke(MethodInvocation invocation) throws Throwable {
-    Objects.requireNonNull(capabilityValidator, "Session validator has not yet been set.");
-
-    // Ensure ROOT is always permitted.
-    ImmutableList.Builder<Capability> whitelistBuilder =
-        ImmutableList.<Capability>builder().add(Capability.ROOT);
-
-    Method method = invocation.getMethod();
-    Requires requires = method.getAnnotation(Requires.class);
-    if (requires != null) {
-      whitelistBuilder.add(requires.whitelist());
-    }
-
-    List<Capability> whitelist = whitelistBuilder.build();
-    LOG.fine("Operation " + method.getName() + " may be performed by: " + whitelist);
-    Optional<SessionKey> sessionKey = FluentIterable.from(Arrays.asList(invocation.getArguments()))
-        .firstMatch(Predicates.instanceOf(SessionKey.class)).transform(CAST);
-    if (!sessionKey.isPresent()) {
-      LOG.severe("Interceptor should only be applied to methods accepting a SessionKey, but "
-          + method + " does not.");
-      return invocation.proceed();
-    }
-
-    String key = capabilityValidator.toString(sessionKey.get());
-    for (Capability user : whitelist) {
-      LOG.fine("Attempting to validate " + key + " against " + user);
-      try {
-        capabilityValidator.checkAuthorized(sessionKey.get(), user, AuditCheck.NONE);
-
-        LOG.info("Permitting " + key + " to act as "
-            + user + " and perform action " + method.getName());
-        return invocation.proceed();
-      } catch (AuthFailedException e) {
-        LOG.fine("Auth failed: " + e);
-      }
-    }
-
-    // User is not permitted to perform this operation.
-    return Responses.addMessage(
-        Responses.empty(),
-        ResponseCode.AUTH_FAILED,
-        "Session identified by '" + key
-            + "' does not have the required capability to perform this action: " + whitelist);
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/thrift/auth/Requires.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/auth/Requires.java b/src/main/java/org/apache/aurora/scheduler/thrift/auth/Requires.java
deleted file mode 100644
index 9b34318..0000000
--- a/src/main/java/org/apache/aurora/scheduler/thrift/auth/Requires.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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.thrift.auth;
-
-import java.lang.annotation.Retention;
-import java.lang.annotation.Target;
-
-import org.apache.aurora.auth.CapabilityValidator.Capability;
-
-import static java.lang.annotation.ElementType.METHOD;
-import static java.lang.annotation.RetentionPolicy.RUNTIME;
-
-/**
- * Annotation applied to a method that may allow users with non-ROOT capabilities to perform
- * an action.
- */
-@Target(METHOD) @Retention(RUNTIME)
-public @interface Requires {
-  /**
-   * The list of capabilities required to perform an action.
-   */
-  Capability[] whitelist() default { Capability.ROOT };
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java b/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java
deleted file mode 100644
index e1a21f6..0000000
--- a/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * 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.thrift.auth;
-
-import java.util.Map;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.inject.AbstractModule;
-import com.google.inject.Module;
-import com.google.inject.TypeLiteral;
-
-import org.apache.aurora.auth.CapabilityValidator;
-import org.apache.aurora.auth.CapabilityValidator.Capability;
-import org.apache.aurora.auth.SessionValidator;
-import org.apache.aurora.auth.UnsecureAuthModule;
-import org.apache.aurora.common.args.Arg;
-import org.apache.aurora.common.args.CmdLine;
-import org.apache.aurora.common.args.constraints.NotEmpty;
-import org.apache.aurora.scheduler.app.MoreModules;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Binding module for authentication of users with special capabilities for admin functions.
- */
-public class ThriftAuthModule extends AbstractModule {
-
-  private static final Map<Capability, String> DEFAULT_CAPABILITIES =
-      ImmutableMap.of(Capability.ROOT, "mesos");
-
-  @NotEmpty
-  @CmdLine(name = "user_capabilities",
-      help = "Concrete name mappings for administration capabilities.")
-  private static final Arg<Map<Capability, String>> USER_CAPABILITIES =
-      Arg.create(DEFAULT_CAPABILITIES);
-
-  @CmdLine(name = "auth_module",
-      help = "A Guice module to provide auth bindings. NOTE: The default is unsecure.")
-  private static final Arg<Module> AUTH_MODULE = Arg.create(new UnsecureAuthModule());
-
-  private static final Iterable<Class<?>> AUTH_MODULE_CLASSES =
-      ImmutableList.of(SessionValidator.class, CapabilityValidator.class);
-
-  private Map<Capability, String> capabilities;
-  private final Module authModule;
-
-  public ThriftAuthModule() {
-    this(USER_CAPABILITIES.get(), AUTH_MODULE.get());
-  }
-
-  @VisibleForTesting
-  public ThriftAuthModule(Map<Capability, String> capabilities, Module authModule) {
-    this.capabilities = requireNonNull(capabilities);
-    this.authModule = requireNonNull(authModule);
-  }
-
-  @Override
-  protected void configure() {
-    Preconditions.checkArgument(
-        capabilities.containsKey(Capability.ROOT),
-        "A ROOT capability must be provided with --user_capabilities");
-    bind(new TypeLiteral<Map<Capability, String>>() { }).toInstance(capabilities);
-
-    requireBinding(SessionValidator.class);
-    requireBinding(CapabilityValidator.class);
-
-    install(MoreModules.wrapInPrivateModule(authModule, AUTH_MODULE_CLASSES));
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/test/java/org/apache/aurora/auth/UnsecureSessionContextTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/auth/UnsecureSessionContextTest.java b/src/test/java/org/apache/aurora/auth/UnsecureSessionContextTest.java
deleted file mode 100644
index c3c3465..0000000
--- a/src/test/java/org/apache/aurora/auth/UnsecureSessionContextTest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * 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.auth;
-
-import java.util.Optional;
-
-import org.apache.aurora.common.testing.easymock.EasyMockTest;
-
-import org.apache.shiro.subject.PrincipalCollection;
-import org.apache.shiro.subject.Subject;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.apache.aurora.auth.UnsecureSessionContext.UNSECURE;
-import static org.easymock.EasyMock.expect;
-import static org.junit.Assert.assertEquals;
-
-public class UnsecureSessionContextTest extends EasyMockTest {
-  private Subject subject;
-  private PrincipalCollection principalCollection;
-
-  @Before
-  public void setUp() {
-    subject = createMock(Subject.class);
-    principalCollection = createMock(PrincipalCollection.class);
-  }
-
-  @Test
-  public void testNonStringPrincipal() {
-    expect(subject.getPrincipals()).andReturn(principalCollection);
-    expect(principalCollection.oneByType(String.class)).andReturn(null);
-
-    control.replay();
-
-    assertEquals(UNSECURE, new UnsecureSessionContext(() -> Optional.of(subject)).getIdentity());
-  }
-
-  @Test
-  public void testEmptySubject() {
-    control.replay();
-
-    assertEquals(UNSECURE, new UnsecureSessionContext(Optional::empty).getIdentity());
-  }
-
-  @Test
-  public void testStringSubject() {
-    expect(subject.getPrincipals()).andReturn(principalCollection);
-    expect(principalCollection.oneByType(String.class)).andReturn("jsmith");
-
-    control.replay();
-
-    assertEquals("jsmith", new UnsecureSessionContext(() -> Optional.of(subject)).getIdentity());
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/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 8af7c46..fb5159d 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,8 +13,6 @@
  */
 package org.apache.aurora.scheduler.http.api;
 
-import java.nio.charset.StandardCharsets;
-
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 
@@ -47,7 +45,6 @@ import org.apache.aurora.gen.RoleSummary;
 import org.apache.aurora.gen.RoleSummaryResult;
 import org.apache.aurora.gen.ScheduleStatusResult;
 import org.apache.aurora.gen.ScheduledTask;
-import org.apache.aurora.gen.SessionKey;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskConstraint;
 import org.apache.aurora.gen.TaskQuery;
@@ -114,20 +111,17 @@ public class ApiBetaTest extends JettyServerModuleTest {
     Lock lock = new Lock()
         .setKey(LockKey.job(new JobKey("role", "env", "name")))
         .setToken("token");
-    SessionKey session = new SessionKey()
-        .setData("session data".getBytes(StandardCharsets.UTF_8))
-        .setMechanism("fake");
     Response response = new Response()
         .setResponseCode(OK);
 
     JobConfiguration job = JOB_CONFIG.newBuilder();
-    expect(thrift.createJob(anyObject(), eq(lock), eq(session))).andReturn(response);
+    expect(thrift.createJob(anyObject(), eq(lock), eq(null))).andReturn(response);
 
     replayAndStart();
 
     Response actualResponse = getRequestBuilder("/apibeta/createJob")
         .entity(
-            ImmutableMap.of("description", job, "lock", lock, "session", session),
+            ImmutableMap.of("description", job, "lock", lock),
             MediaType.APPLICATION_JSON)
         .post(Response.class);
     assertEquals(IResponse.build(response), IResponse.build(actualResponse));

http://git-wip-us.apache.org/repos/asf/aurora/blob/4eeec7a7/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java
index 0f8748d..9c8460c 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java
@@ -19,6 +19,7 @@ import org.apache.aurora.common.testing.easymock.EasyMockTest;
 import org.apache.shiro.subject.Subject;
 import org.junit.Test;
 
+import static org.apache.aurora.scheduler.thrift.AuditMessages.DEFAULT_USER;
 import static org.easymock.EasyMock.expect;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertThat;
@@ -30,9 +31,9 @@ public class AuditMessagesTest extends EasyMockTest {
 
     control.replay();
 
-    assertThat(emptyMessages.killedBy("legacy").get(), containsString("legacy"));
-    assertThat(emptyMessages.restartedBy("legacy").get(), containsString("legacy"));
-    assertThat(emptyMessages.transitionedBy("legacy").get(), containsString("legacy"));
+    assertThat(emptyMessages.killedByRemoteUser().get(), containsString(DEFAULT_USER));
+    assertThat(emptyMessages.restartedByRemoteUser().get(), containsString(DEFAULT_USER));
+    assertThat(emptyMessages.transitionedBy().get(), containsString(DEFAULT_USER));
   }
 
   @Test
@@ -44,8 +45,8 @@ public class AuditMessagesTest extends EasyMockTest {
 
     control.replay();
 
-    assertThat(presentMessages.killedBy("legacy").get(), containsString("shiro"));
-    assertThat(presentMessages.restartedBy("legacy").get(), containsString("shiro"));
-    assertThat(presentMessages.transitionedBy("legacy").get(), containsString("shiro"));
+    assertThat(presentMessages.killedByRemoteUser().get(), containsString("shiro"));
+    assertThat(presentMessages.restartedByRemoteUser().get(), containsString("shiro"));
+    assertThat(presentMessages.transitionedBy().get(), containsString("shiro"));
   }
 }