You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2015/11/30 18:33:46 UTC

[2/2] aurora git commit: Remove SessionKey from APIs and implementations.

Remove SessionKey from APIs and implementations.

Since our auth* is now transport-level, we don't need it in the app-level
messages.

Bugs closed: AURORA-814

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


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

Branch: refs/heads/master
Commit: 7b1c2b2fce3466ee8cb7cb035796ded6615a2658
Parents: 8524dbf
Author: Bill Farner <wf...@apache.org>
Authored: Mon Nov 30 09:33:37 2015 -0800
Committer: Bill Farner <wf...@apache.org>
Committed: Mon Nov 30 09:33:37 2015 -0800

----------------------------------------------------------------------
 .../thrift/org/apache/aurora/gen/api.thrift     |  75 +++---
 .../thrift/SchedulerThriftInterface.java        | 158 ++++--------
 .../thrift/aop/AnnotatedAuroraAdmin.java        |  46 ++--
 .../aurora/client/api/scheduler_client.py       |   8 +-
 .../aurora/scheduler/http/api/ApiBetaTest.java  |   2 +-
 .../http/api/security/HttpSecurityIT.java       |  36 +--
 .../ShiroAuthorizingInterceptorTest.java        |   3 +-
 .../ShiroAuthorizingParamInterceptorTest.java   |  10 +-
 .../thrift/SchedulerThriftInterfaceTest.java    | 254 +++++++++----------
 .../aurora/scheduler/thrift/ThriftIT.java       |   2 +-
 .../scheduler/thrift/aop/AopModuleTest.java     |   8 +-
 .../scheduler/thrift/aop/ForwardingThrift.java  | 163 +++++-------
 .../thrift/aop/LoggingInterceptorTest.java      |   5 -
 .../apache/aurora/client/api/test_restarter.py  |  16 +-
 .../aurora/client/api/test_scheduler_client.py  |  96 +++----
 .../apache/aurora/client/api/test_updater.py    |  22 +-
 .../aurora/client/fake_scheduler_proxy.py       |   6 +-
 17 files changed, 354 insertions(+), 556 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/api/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/api/src/main/thrift/org/apache/aurora/gen/api.thrift b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
index b262884..d765dd7 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -51,10 +51,6 @@ struct Identity {
   2: string user
 }
 
-/** Deprecated, to be removed in 0.11.0 (https://issues.apache.org/jira/browse/AURORA-1229). */
-struct SessionKey {
-}
-
 struct ResourceAggregate {
   /** Number of CPU cores allotted. */
   1: double numCpus
@@ -1016,73 +1012,66 @@ service ReadOnlyScheduler {
   Response getJobUpdateDiff(1: JobUpdateRequest request)
 }
 
-// Due to assumptions in the client all authenticated RPCs must have a SessionKey as their
-// last argument. Note that the order in this file is what matters, and message numbers should still
-// never be reused.
 service AuroraSchedulerManager extends ReadOnlyScheduler {
   /**
    * Creates a new job.  The request will be denied if a job with the provided name already exists
    * in the cluster.
    */
-  Response createJob(1: JobConfiguration description, 3: Lock lock, 2: SessionKey session)
+  Response createJob(1: JobConfiguration description, 3: Lock lock)
 
   /**
    * Enters a job into the cron schedule, without actually starting the job.
    * If the job is already present in the schedule, this will update the schedule entry with the new
    * configuration.
    */
-  Response scheduleCronJob(1: JobConfiguration description, 3: Lock lock, 2: SessionKey session)
+  Response scheduleCronJob(1: JobConfiguration description, 3: Lock lock)
 
   /**
    * Removes a job from the cron schedule. The request will be denied if the job was not previously
    * scheduled with scheduleCronJob.
    */
-  Response descheduleCronJob(4: JobKey job, 3: Lock lock, 2: SessionKey session)
+  Response descheduleCronJob(4: JobKey job, 3: Lock lock)
 
   /**
    * Starts a cron job immediately.  The request will be denied if the specified job does not
    * exist for the role account, or the job is not a cron job.
    */
-  Response startCronJob(4: JobKey job, 3: SessionKey session)
+  Response startCronJob(4: JobKey job)
 
   /** Restarts a batch of shards. */
-  Response restartShards(5: JobKey job, 3: set<i32> shardIds, 6: Lock lock 4: SessionKey session)
+  Response restartShards(5: JobKey job, 3: set<i32> shardIds, 6: Lock lock)
 
   /** Initiates a kill on tasks. */
-  Response killTasks(1: TaskQuery query, 3: Lock lock, 2: SessionKey session)
+  Response killTasks(1: TaskQuery query, 3: Lock lock)
 
   /**
    * Adds new instances specified by the AddInstancesConfig. A job represented by the JobKey must be
    * protected by Lock.
    */
-  Response addInstances(
-      1: AddInstancesConfig config,
-      2: Lock lock,
-      3: SessionKey session)
+  Response addInstances(1: AddInstancesConfig config, 2: Lock lock)
 
   /**
    * Creates and saves a new Lock instance guarding against multiple mutating operations within the
    * context defined by LockKey.
    */
-  Response acquireLock(1: LockKey lockKey, 2: SessionKey session)
+  Response acquireLock(1: LockKey lockKey)
 
   /** Releases the lock acquired earlier in acquireLock call. */
-  Response releaseLock(1: Lock lock, 2: LockValidation validation, 3: SessionKey session)
+  Response releaseLock(1: Lock lock, 2: LockValidation validation)
 
   // TODO(maxim): reevaluate if it's still needed when client updater is gone (AURORA-785).
   /**
    * Replaces the template (configuration) for the existing cron job.
    * The cron job template (configuration) must exist for the call to succeed.
    */
-  Response replaceCronTemplate(1: JobConfiguration config, 2: Lock lock, 3: SessionKey session)
+  Response replaceCronTemplate(1: JobConfiguration config, 2: Lock lock)
 
   /** Starts update of the existing service job. */
   Response startJobUpdate(
       /** A description of how to change the job. */
       1: JobUpdateRequest request,
       /** A user-specified message to include with the induced job update state change. */
-      3: string message,
-      2: SessionKey session)
+      3: string message)
 
   /**
    * Pauses the specified job update. Can be resumed by resumeUpdate call.
@@ -1091,31 +1080,28 @@ service AuroraSchedulerManager extends ReadOnlyScheduler {
       /** The update to pause. */
       1: JobUpdateKey key,
       /** A user-specified message to include with the induced job update state change. */
-      3: string message,
-      2: SessionKey session)
+      3: string message)
 
   /** Resumes progress of a previously paused job update. */
   Response resumeJobUpdate(
       /** The update to resume. */
       1: JobUpdateKey key,
       /** A user-specified message to include with the induced job update state change. */
-      3: string message,
-      2: SessionKey session)
+      3: string message)
 
   /** Permanently aborts the job update. Does not remove the update history. */
   Response abortJobUpdate(
       /** The update to abort. */
       1: JobUpdateKey key,
       /** A user-specified message to include with the induced job update state change. */
-      3: string message,
-      2: SessionKey session)
+      3: string message)
 
   /**
    * Allows progress of the job update in case blockIfNoPulsesAfterMs is specified in
    * JobUpdateSettings. Unblocks progress if the update was previously blocked.
    * Responds with ResponseCode.INVALID_REQUEST in case an unknown update key is specified.
    */
-  Response pulseJobUpdate(1: JobUpdateKey key, 2: SessionKey session)
+  Response pulseJobUpdate(1: JobUpdateKey key)
 }
 
 struct InstanceConfigRewrite {
@@ -1147,7 +1133,7 @@ struct RewriteConfigsRequest {
 // https://issues.apache.org/jira/browse/THRIFT-66 is resolved.
 service AuroraAdmin extends AuroraSchedulerManager {
   /** Assign quota to a user.  This will overwrite any pre-existing quota for the user. */
-  Response setQuota(1: string ownerRole, 2: ResourceAggregate quota, 3: SessionKey session)
+  Response setQuota(1: string ownerRole, 2: ResourceAggregate quota)
 
   /**
    * Forces a task into a specific state.  This does not guarantee the task will enter the given
@@ -1156,44 +1142,43 @@ service AuroraAdmin extends AuroraSchedulerManager {
    */
   Response forceTaskState(
       1: string taskId,
-      2: ScheduleStatus status,
-      3: SessionKey session)
+      2: ScheduleStatus status)
 
   /** Immediately writes a storage snapshot to disk. */
-  Response performBackup(1: SessionKey session)
+  Response performBackup()
 
   /** Lists backups that are available for recovery. */
-  Response listBackups(1: SessionKey session)
+  Response listBackups()
 
   /** Loads a backup to an in-memory storage.  This must precede all other recovery operations. */
-  Response stageRecovery(1: string backupId, 2: SessionKey session)
+  Response stageRecovery(1: string backupId)
 
   /** Queries for tasks in a staged recovery. */
-  Response queryRecovery(1: TaskQuery query, 2: SessionKey session)
+  Response queryRecovery(1: TaskQuery query)
 
   /** Deletes tasks from a staged recovery. */
-  Response deleteRecoveryTasks(1: TaskQuery query, 2: SessionKey session)
+  Response deleteRecoveryTasks(1: TaskQuery query)
 
   /** Commits a staged recovery, completely replacing the previous storage state. */
-  Response commitRecovery(1: SessionKey session)
+  Response commitRecovery()
 
   /** Unloads (aborts) a staged recovery. */
-  Response unloadRecovery(1: SessionKey session)
+  Response unloadRecovery()
 
   /** Put the given hosts into maintenance mode. */
-  Response startMaintenance(1: Hosts hosts, 2: SessionKey session)
+  Response startMaintenance(1: Hosts hosts)
 
   /** Ask scheduler to begin moving tasks scheduled on given hosts. */
-  Response drainHosts(1: Hosts hosts, 2: SessionKey session)
+  Response drainHosts(1: Hosts hosts)
 
   /** Retrieve the current maintenance states for a group of hosts. */
-  Response maintenanceStatus(1: Hosts hosts, 2: SessionKey session)
+  Response maintenanceStatus(1: Hosts hosts)
 
   /** Set the given hosts back into serving mode. */
-  Response endMaintenance(1: Hosts hosts, 2: SessionKey session)
+  Response endMaintenance(1: Hosts hosts)
 
   /** Start a storage snapshot and block until it completes. */
-  Response snapshot(1: SessionKey session)
+  Response snapshot()
 
   /**
    * Forcibly rewrites the stored definition of user configurations.  This is intended to be used
@@ -1202,5 +1187,5 @@ service AuroraAdmin extends AuroraSchedulerManager {
    * The scheduler may do some validation of the rewritten configurations, but it is important
    * that the caller take care to provide valid input and alter only necessary fields.
    */
-  Response rewriteConfigs(1: RewriteConfigsRequest request, 2: SessionKey session)
+  Response rewriteConfigs(1: RewriteConfigsRequest request)
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/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 ccabf55..326d4fb 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -67,7 +67,6 @@ import org.apache.aurora.gen.Response;
 import org.apache.aurora.gen.Result;
 import org.apache.aurora.gen.RewriteConfigsRequest;
 import org.apache.aurora.gen.ScheduleStatus;
-import org.apache.aurora.gen.SessionKey;
 import org.apache.aurora.gen.StartJobUpdateResult;
 import org.apache.aurora.gen.StartMaintenanceResult;
 import org.apache.aurora.gen.TaskQuery;
@@ -219,12 +218,8 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response createJob(
-      JobConfiguration mutableJob,
-      @Nullable final Lock mutableLock,
-      @Nullable SessionKey ignored) {
-
-    final SanitizedConfiguration sanitized;
+  public Response createJob(JobConfiguration mutableJob, @Nullable Lock mutableLock) {
+    SanitizedConfiguration sanitized;
     try {
       sanitized = SanitizedConfiguration.fromUnsanitized(IJobConfiguration.build(mutableJob));
     } catch (TaskDescriptionException e) {
@@ -238,7 +233,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
     return storage.write(new MutateWork.Quiet<Response>() {
       @Override
       public Response apply(MutableStoreProvider storeProvider) {
-        final IJobConfiguration job = sanitized.getJobConfig();
+        IJobConfiguration job = sanitized.getJobConfig();
 
         try {
           lockManager.validateIfLocked(
@@ -287,13 +282,13 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
 
   private Response createOrUpdateCronTemplate(
       JobConfiguration mutableJob,
-      @Nullable final Lock mutableLock,
-      final boolean updateOnly) {
+      @Nullable Lock mutableLock,
+      boolean updateOnly) {
 
     IJobConfiguration job = IJobConfiguration.build(mutableJob);
-    final IJobKey jobKey = JobKeys.assertValid(job.getKey());
+    IJobKey jobKey = JobKeys.assertValid(job.getKey());
 
-    final SanitizedConfiguration sanitized;
+    SanitizedConfiguration sanitized;
     try {
       sanitized = SanitizedConfiguration.fromUnsanitized(job);
     } catch (TaskDescriptionException e) {
@@ -340,29 +335,17 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response scheduleCronJob(
-      JobConfiguration mutableJob,
-      @Nullable Lock mutableLock,
-      @Nullable SessionKey ignored) {
-
+  public Response scheduleCronJob(JobConfiguration mutableJob, @Nullable Lock mutableLock) {
     return createOrUpdateCronTemplate(mutableJob, mutableLock, false);
   }
 
   @Override
-  public Response replaceCronTemplate(
-      JobConfiguration mutableJob,
-      @Nullable Lock mutableLock,
-      @Nullable SessionKey ignored) {
-
+  public Response replaceCronTemplate(JobConfiguration mutableJob, @Nullable Lock mutableLock) {
     return createOrUpdateCronTemplate(mutableJob, mutableLock, true);
   }
 
   @Override
-  public Response descheduleCronJob(
-      JobKey mutableJobKey,
-      @Nullable Lock mutableLock,
-      @Nullable SessionKey ignored) {
-
+  public Response descheduleCronJob(JobKey mutableJobKey, @Nullable Lock mutableLock) {
     try {
       IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJobKey));
       lockManager.validateIfLocked(
@@ -384,7 +367,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response startCronJob(JobKey mutableJobKey, @Nullable SessionKey ignored) {
+  public Response startCronJob(JobKey mutableJobKey) {
     IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJobKey));
 
     try {
@@ -408,15 +391,11 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
 
   @Override
   public Response getPendingReason(TaskQuery query) throws TException {
-
-    // Only PENDING tasks should be considered.
-
     return readOnlyScheduler.getPendingReason(query);
   }
 
   @Override
   public Response getConfigSummary(JobKey job) throws TException {
-
     return readOnlyScheduler.getConfigSummary(job);
   }
 
@@ -460,18 +439,14 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response killTasks(
-      TaskQuery mutableQuery,
-      final Lock mutableLock,
-      @Nullable SessionKey ignored) {
-
+  public Response killTasks(TaskQuery mutableQuery, Lock mutableLock) {
     requireNonNull(mutableQuery);
 
     if (mutableQuery.getJobName() != null && WHITESPACE.matchesAllOf(mutableQuery.getJobName())) {
       return invalidRequest(String.format("Invalid job name: '%s'", mutableQuery.getJobName()));
     }
 
-    final Query.Builder query = implicitKillQuery(mutableQuery);
+    Query.Builder query = implicitKillQuery(mutableQuery);
     Preconditions.checkState(
         !query.get().isSetOwner(),
         "The owner field in a query should have been unset by Query.Builder.");
@@ -510,11 +485,10 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   @Override
   public Response restartShards(
       JobKey mutableJobKey,
-      final Set<Integer> shardIds,
-      @Nullable final Lock mutableLock,
-      @Nullable SessionKey ignored) {
+      Set<Integer> shardIds,
+      @Nullable Lock mutableLock) {
 
-    final IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJobKey));
+    IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJobKey));
     checkNotBlank(shardIds);
 
     return storage.write(new MutateWork.Quiet<Response>() {
@@ -529,8 +503,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
         }
 
         Query.Builder query = Query.instanceScoped(jobKey, shardIds).active();
-        final Iterable<IScheduledTask> matchingTasks =
-            storeProvider.getTaskStore().fetchTasks(query);
+        Iterable<IScheduledTask> matchingTasks = storeProvider.getTaskStore().fetchTasks(query);
         if (Iterables.size(matchingTasks) != shardIds.size()) {
           return invalidRequest("Not all requested shards are active.");
         }
@@ -560,11 +533,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response setQuota(
-      final String ownerRole,
-      final ResourceAggregate resourceAggregate,
-      @Nullable SessionKey ignored) {
-
+  public Response setQuota(String ownerRole, ResourceAggregate resourceAggregate) {
     checkNotBlank(ownerRole);
     requireNonNull(resourceAggregate);
 
@@ -585,37 +554,33 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response startMaintenance(Hosts hosts, @Nullable SessionKey ignored) {
+  public Response startMaintenance(Hosts hosts) {
     return ok(Result.startMaintenanceResult(
         new StartMaintenanceResult()
             .setStatuses(maintenance.startMaintenance(hosts.getHostNames()))));
   }
 
   @Override
-  public Response drainHosts(Hosts hosts, @Nullable SessionKey ignored) {
+  public Response drainHosts(Hosts hosts) {
     return ok(Result.drainHostsResult(
         new DrainHostsResult().setStatuses(maintenance.drain(hosts.getHostNames()))));
   }
 
   @Override
-  public Response maintenanceStatus(Hosts hosts, @Nullable SessionKey ignored) {
+  public Response maintenanceStatus(Hosts hosts) {
     return ok(Result.maintenanceStatusResult(
         new MaintenanceStatusResult().setStatuses(maintenance.getStatus(hosts.getHostNames()))));
   }
 
   @Override
-  public Response endMaintenance(Hosts hosts, @Nullable SessionKey ignored) {
+  public Response endMaintenance(Hosts hosts) {
     return ok(Result.endMaintenanceResult(
         new EndMaintenanceResult()
             .setStatuses(maintenance.endMaintenance(hosts.getHostNames()))));
   }
 
   @Override
-  public Response forceTaskState(
-      final String taskId,
-      final ScheduleStatus status,
-      @Nullable SessionKey ignored) {
-
+  public Response forceTaskState(String taskId, ScheduleStatus status) {
     checkNotBlank(taskId);
     requireNonNull(status);
 
@@ -635,58 +600,55 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response performBackup(@Nullable SessionKey ignored) {
+  public Response performBackup() {
     backup.backupNow();
     return ok();
   }
 
   @Override
-  public Response listBackups(@Nullable SessionKey ignored) {
+  public Response listBackups() {
     return ok(Result.listBackupsResult(new ListBackupsResult()
         .setBackups(recovery.listBackups())));
   }
 
   @Override
-  public Response stageRecovery(String backupId, @Nullable SessionKey ignored) {
+  public Response stageRecovery(String backupId) {
     recovery.stage(backupId);
     return ok();
   }
 
   @Override
-  public Response queryRecovery(TaskQuery query, @Nullable SessionKey ignored) {
+  public Response queryRecovery(TaskQuery query) {
     return ok(Result.queryRecoveryResult(new QueryRecoveryResult()
         .setTasks(IScheduledTask.toBuildersSet(recovery.query(Query.arbitrary(query))))));
   }
 
   @Override
-  public Response deleteRecoveryTasks(TaskQuery query, @Nullable SessionKey ignored) {
+  public Response deleteRecoveryTasks(TaskQuery query) {
     recovery.deleteTasks(Query.arbitrary(query));
     return ok();
   }
 
   @Override
-  public Response commitRecovery(@Nullable SessionKey ignored) {
+  public Response commitRecovery() {
     recovery.commit();
     return ok();
   }
 
   @Override
-  public Response unloadRecovery(@Nullable SessionKey ignored) {
+  public Response unloadRecovery() {
     recovery.unload();
     return ok();
   }
 
   @Override
-  public Response snapshot(@Nullable SessionKey ignored) {
+  public Response snapshot() {
     storage.snapshot();
     return ok();
   }
 
   @Override
-  public Response rewriteConfigs(
-      final RewriteConfigsRequest request,
-      @Nullable SessionKey ignored) {
-
+  public Response rewriteConfigs(RewriteConfigsRequest request) {
     if (request.getRewriteCommandsSize() == 0) {
       return addMessage(Responses.empty(), INVALID_REQUEST, "No rewrite commands provided.");
     }
@@ -803,16 +765,12 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response addInstances(
-      final AddInstancesConfig config,
-      @Nullable final Lock mutableLock,
-      @Nullable SessionKey ignored) {
-
+  public Response addInstances(AddInstancesConfig config, @Nullable Lock mutableLock) {
     requireNonNull(config);
     checkNotBlank(config.getInstanceIds());
-    final IJobKey jobKey = JobKeys.assertValid(IJobKey.build(config.getKey()));
+    IJobKey jobKey = JobKeys.assertValid(IJobKey.build(config.getKey()));
 
-    final ITaskConfig task;
+    ITaskConfig task;
     try {
       task = ConfigurationManager.validateAndPopulate(
           ITaskConfig.build(config.getTaskConfig()));
@@ -860,13 +818,13 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
     });
   }
 
-  public Optional<IJobConfiguration> getCronJob(StoreProvider storeProvider, final IJobKey jobKey) {
+  public Optional<IJobConfiguration> getCronJob(StoreProvider storeProvider, IJobKey jobKey) {
     requireNonNull(jobKey);
     return storeProvider.getCronJobStore().fetchJob(jobKey);
   }
 
   @Override
-  public Response acquireLock(LockKey mutableLockKey, @Nullable SessionKey ignored) {
+  public Response acquireLock(LockKey mutableLockKey) {
     requireNonNull(mutableLockKey);
 
     ILockKey lockKey = ILockKey.build(mutableLockKey);
@@ -881,11 +839,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response releaseLock(
-      Lock mutableLock,
-      LockValidation validation,
-      @Nullable SessionKey ignored) {
-
+  public Response releaseLock(Lock mutableLock, LockValidation validation) {
     requireNonNull(mutableLock);
     requireNonNull(validation);
 
@@ -957,15 +911,11 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response startJobUpdate(
-      JobUpdateRequest mutableRequest,
-      @Nullable final String message,
-      @Nullable SessionKey ignored) {
-
+  public Response startJobUpdate(JobUpdateRequest mutableRequest, @Nullable String message) {
     requireNonNull(mutableRequest);
 
     // TODO(maxim): Switch to key field instead when AURORA-749 is fixed.
-    final IJobKey job = JobKeys.assertValid(IJobKey.build(new JobKey()
+    IJobKey job = JobKeys.assertValid(IJobKey.build(new JobKey()
         .setRole(mutableRequest.getTaskConfig().getOwner().getRole())
         .setEnvironment(mutableRequest.getTaskConfig().getEnvironment())
         .setName(mutableRequest.getTaskConfig().getJobName())));
@@ -1000,7 +950,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
       return invalidRequest(INVALID_PULSE_TIMEOUT);
     }
 
-    final IJobUpdateRequest request;
+    IJobUpdateRequest request;
     try {
       request = IJobUpdateRequest.build(new JobUpdateRequest(mutableRequest).setTaskConfig(
           ConfigurationManager.validateAndPopulate(
@@ -1077,10 +1027,10 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
 
   private Response changeJobUpdateState(
       JobUpdateKey mutableKey,
-      final JobUpdateStateChange change,
-      final Optional<String> message) {
+      JobUpdateStateChange change,
+      Optional<String> message) {
 
-    final IJobUpdateKey key = IJobUpdateKey.build(mutableKey);
+    IJobUpdateKey key = IJobUpdateKey.build(mutableKey);
     JobKeys.assertValid(key.getJob());
     return storage.write(new MutateWork.Quiet<Response>() {
       @Override
@@ -1104,11 +1054,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response pauseJobUpdate(
-      JobUpdateKey mutableKey,
-      @Nullable String message,
-      @Nullable SessionKey ignored) {
-
+  public Response pauseJobUpdate(JobUpdateKey mutableKey, @Nullable String message) {
     return changeJobUpdateState(
         mutableKey,
         JobUpdateController::pause,
@@ -1116,11 +1062,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response resumeJobUpdate(
-      JobUpdateKey mutableKey,
-      @Nullable String message,
-      @Nullable SessionKey ignored) {
-
+  public Response resumeJobUpdate(JobUpdateKey mutableKey, @Nullable String message) {
     return changeJobUpdateState(
         mutableKey,
         JobUpdateController::resume,
@@ -1128,11 +1070,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response abortJobUpdate(
-      JobUpdateKey mutableKey,
-      @Nullable String message,
-      @Nullable SessionKey ignored) {
-
+  public Response abortJobUpdate(JobUpdateKey mutableKey, @Nullable String message) {
     return changeJobUpdateState(
         mutableKey,
         JobUpdateController::abort,
@@ -1140,7 +1078,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response pulseJobUpdate(JobUpdateKey mutableUpdateKey, @Nullable SessionKey ignored) {
+  public Response pulseJobUpdate(JobUpdateKey mutableUpdateKey) {
     IJobUpdateKey updateKey = validateJobUpdateKey(mutableUpdateKey);
     try {
       JobUpdatePulseStatus result = jobUpdateController.pulse(updateKey);

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java b/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java
index 6d30c25..f6669ef 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java
@@ -27,7 +27,6 @@ import org.apache.aurora.gen.Lock;
 import org.apache.aurora.gen.LockKey;
 import org.apache.aurora.gen.LockValidation;
 import org.apache.aurora.gen.Response;
-import org.apache.aurora.gen.SessionKey;
 import org.apache.aurora.gen.TaskQuery;
 import org.apache.aurora.scheduler.http.api.security.AuthorizingParam;
 import org.apache.thrift.TException;
@@ -47,88 +46,73 @@ public interface AnnotatedAuroraAdmin extends AuroraAdmin.Iface {
   @Override
   Response createJob(
       @AuthorizingParam @Nullable JobConfiguration description,
-      @Nullable Lock lock,
-      @Nullable SessionKey session) throws TException;
+      @Nullable Lock lock) throws TException;
 
   @Override
   Response scheduleCronJob(
       @AuthorizingParam @Nullable JobConfiguration description,
-      @Nullable Lock lock,
-      @Nullable SessionKey session) throws TException;
+      @Nullable Lock lock) throws TException;
 
   @Override
   Response descheduleCronJob(
       @AuthorizingParam @Nullable JobKey job,
-      @Nullable Lock lock,
-      @Nullable SessionKey session) throws TException;
+      @Nullable Lock lock) throws TException;
 
   @Override
   Response startCronJob(
-      @AuthorizingParam @Nullable JobKey job,
-      @Nullable SessionKey session) throws TException;
+      @AuthorizingParam @Nullable JobKey job) throws TException;
 
   @Override
   Response restartShards(
       @AuthorizingParam @Nullable JobKey job,
       @Nullable Set<Integer> shardIds,
-      @Nullable Lock lock,
-      @Nullable SessionKey session) throws TException;
+      @Nullable Lock lock) throws TException;
 
   @Override
   Response killTasks(
       @AuthorizingParam @Nullable TaskQuery query,
-      @Nullable Lock lock,
-      @Nullable SessionKey session) throws TException;
+      @Nullable Lock lock) throws TException;
 
   @Override
   Response addInstances(
       @AuthorizingParam @Nullable AddInstancesConfig config,
-      @Nullable Lock lock,
-      @Nullable SessionKey session) throws TException;
+      @Nullable Lock lock) throws TException;
 
   @Override
   Response acquireLock(
-      @AuthorizingParam @Nullable LockKey lockKey,
-      @Nullable SessionKey session) throws TException;
+      @AuthorizingParam @Nullable LockKey lockKey) throws TException;
 
   @Override
   Response releaseLock(
       @AuthorizingParam @Nullable Lock lock,
-      @Nullable LockValidation validation,
-      @Nullable SessionKey session) throws TException;
+      @Nullable LockValidation validation) throws TException;
 
   @Override
   Response replaceCronTemplate(
       @AuthorizingParam @Nullable JobConfiguration config,
-      @Nullable Lock lock,
-      @Nullable SessionKey session) throws TException;
+      @Nullable Lock lock) throws TException;
 
   @Override
   Response startJobUpdate(
       @AuthorizingParam @Nullable JobUpdateRequest request,
-      @Nullable String message,
-      @Nullable SessionKey session) throws TException;
+      @Nullable String message) throws TException;
 
   @Override
   Response pauseJobUpdate(
       @AuthorizingParam @Nullable JobUpdateKey key,
-      @Nullable String message,
-      @Nullable SessionKey session) throws TException;
+      @Nullable String message) throws TException;
 
   @Override
   Response resumeJobUpdate(
       @AuthorizingParam @Nullable JobUpdateKey key,
-      @Nullable String message,
-      @Nullable SessionKey session) throws TException;
+      @Nullable String message) throws TException;
 
   @Override
   Response abortJobUpdate(
       @AuthorizingParam @Nullable JobUpdateKey key,
-      @Nullable String message,
-      @Nullable SessionKey session) throws TException;
+      @Nullable String message) throws TException;
 
   @Override
   Response pulseJobUpdate(
-      @AuthorizingParam @Nullable JobUpdateKey key,
-      @Nullable SessionKey session) throws TException;
+      @AuthorizingParam @Nullable JobUpdateKey key) throws TException;
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/src/main/python/apache/aurora/client/api/scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/scheduler_client.py b/src/main/python/apache/aurora/client/api/scheduler_client.py
index c973da1..8636624 100644
--- a/src/main/python/apache/aurora/client/api/scheduler_client.py
+++ b/src/main/python/apache/aurora/client/api/scheduler_client.py
@@ -30,8 +30,8 @@ from apache.aurora.common.auth.auth_module_manager import get_auth_handler
 from apache.aurora.common.cluster import Cluster
 from apache.aurora.common.transport import TRequestsTransport
 
-from gen.apache.aurora.api import AuroraAdmin, ReadOnlyScheduler
-from gen.apache.aurora.api.ttypes import ResponseCode, SessionKey
+from gen.apache.aurora.api import AuroraAdmin
+from gen.apache.aurora.api.ttypes import ResponseCode
 
 try:
   from urlparse import urljoin
@@ -289,14 +289,12 @@ class SchedulerProxy(object):
         while not self._terminating.is_set() and (
             time.time() - start) < self.RPC_MAXIMUM_WAIT.as_(Time.SECONDS):
 
-          # Only automatically append a SessionKey if this is not part of the read-only API.
-          auth_args = () if hasattr(ReadOnlyScheduler.Iface, method_name) else (SessionKey(),)
           try:
             method = getattr(self.client(), method_name)
             if not callable(method):
               return method
 
-            resp = method(*(args + auth_args))
+            resp = method(*args)
             if resp is not None and resp.responseCode == ResponseCode.ERROR_TRANSIENT:
               raise self.TransientError(", ".join(
                   [m.message for m in resp.details] if resp.details else []))

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/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 fb5159d..6dc65b5 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
@@ -115,7 +115,7 @@ public class ApiBetaTest extends JettyServerModuleTest {
         .setResponseCode(OK);
 
     JobConfiguration job = JOB_CONFIG.newBuilder();
-    expect(thrift.createJob(anyObject(), eq(lock), eq(null))).andReturn(response);
+    expect(thrift.createJob(anyObject(), eq(lock))).andReturn(response);
 
     replayAndStart();
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
index 2454959..23db909 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
@@ -190,7 +190,7 @@ public class HttpSecurityIT extends JettyServerModuleTest {
 
   private void assertKillTasksFails(AuroraAdmin.Client client) throws TException {
     try {
-      client.killTasks(null, null, null);
+      client.killTasks(null, null);
       fail("killTasks should fail.");
     } catch (TTransportException e) {
       // Expected.
@@ -199,43 +199,43 @@ public class HttpSecurityIT extends JettyServerModuleTest {
 
   @Test
   public void testAuroraSchedulerManager() throws TException, IOException {
-    expect(auroraAdmin.killTasks(null, new Lock().setMessage("1"), null)).andReturn(OK);
-    expect(auroraAdmin.killTasks(null, new Lock().setMessage("2"), null)).andReturn(OK);
+    expect(auroraAdmin.killTasks(null, new Lock().setMessage("1"))).andReturn(OK);
+    expect(auroraAdmin.killTasks(null, new Lock().setMessage("2"))).andReturn(OK);
 
     TaskQuery jobScopedQuery = Query.jobScoped(JobKeys.from("role", "env", "name")).get();
     TaskQuery adsScopedQuery = Query.jobScoped(ADS_STAGING_JOB).get();
-    expect(auroraAdmin.killTasks(adsScopedQuery, null, null)).andReturn(OK);
+    expect(auroraAdmin.killTasks(adsScopedQuery, null)).andReturn(OK);
 
     replayAndStart();
 
     assertEquals(OK,
-        getAuthenticatedClient(WFARNER).killTasks(null, new Lock().setMessage("1"), null));
+        getAuthenticatedClient(WFARNER).killTasks(null, new Lock().setMessage("1")));
     assertEquals(OK,
-        getAuthenticatedClient(ROOT).killTasks(null, new Lock().setMessage("2"), null));
+        getAuthenticatedClient(ROOT).killTasks(null, new Lock().setMessage("2")));
     assertEquals(
         ResponseCode.INVALID_REQUEST,
-        getAuthenticatedClient(UNPRIVILEGED).killTasks(null, null, null).getResponseCode());
+        getAuthenticatedClient(UNPRIVILEGED).killTasks(null, null).getResponseCode());
     assertEquals(
         ResponseCode.AUTH_FAILED,
         getAuthenticatedClient(UNPRIVILEGED)
-            .killTasks(jobScopedQuery, null, null)
+            .killTasks(jobScopedQuery, null)
             .getResponseCode());
     assertEquals(
         ResponseCode.INVALID_REQUEST,
-        getAuthenticatedClient(BACKUP_SERVICE).killTasks(null, null, null).getResponseCode());
+        getAuthenticatedClient(BACKUP_SERVICE).killTasks(null, null).getResponseCode());
     assertEquals(
         ResponseCode.AUTH_FAILED,
         getAuthenticatedClient(BACKUP_SERVICE)
-            .killTasks(jobScopedQuery, null, null)
+            .killTasks(jobScopedQuery, null)
             .getResponseCode());
     assertEquals(
         ResponseCode.AUTH_FAILED,
         getAuthenticatedClient(DEPLOY_SERVICE)
-            .killTasks(jobScopedQuery, null, null)
+            .killTasks(jobScopedQuery, null)
             .getResponseCode());
     assertEquals(
         OK,
-        getAuthenticatedClient(DEPLOY_SERVICE).killTasks(adsScopedQuery, null, null));
+        getAuthenticatedClient(DEPLOY_SERVICE).killTasks(adsScopedQuery, null));
 
     assertKillTasksFails(getUnauthenticatedClient());
     assertKillTasksFails(getAuthenticatedClient(INCORRECT));
@@ -244,7 +244,7 @@ public class HttpSecurityIT extends JettyServerModuleTest {
 
   private void assertSnapshotFails(AuroraAdmin.Client client) throws TException {
     try {
-      client.snapshot(null);
+      client.snapshot();
       fail("snapshot should fail");
     } catch (TTransportException e) {
       // Expected.
@@ -253,12 +253,12 @@ public class HttpSecurityIT extends JettyServerModuleTest {
 
   @Test
   public void testAuroraAdmin() throws TException {
-    expect(auroraAdmin.snapshot(null)).andReturn(OK);
-    expect(auroraAdmin.listBackups(null)).andReturn(OK);
+    expect(auroraAdmin.snapshot()).andReturn(OK);
+    expect(auroraAdmin.listBackups()).andReturn(OK);
 
     replayAndStart();
 
-    assertEquals(OK, getAuthenticatedClient(ROOT).snapshot(null));
+    assertEquals(OK, getAuthenticatedClient(ROOT).snapshot());
 
     for (Credentials credentials : INVALID_CREDENTIALS) {
       assertSnapshotFails(getAuthenticatedClient(credentials));
@@ -267,10 +267,10 @@ public class HttpSecurityIT extends JettyServerModuleTest {
     for (Credentials credentials : Sets.difference(VALID_CREDENTIALS, ImmutableSet.of(ROOT))) {
       assertEquals(
           ResponseCode.AUTH_FAILED,
-          getAuthenticatedClient(credentials).snapshot(null).getResponseCode());
+          getAuthenticatedClient(credentials).snapshot().getResponseCode());
     }
 
-    assertEquals(OK, getAuthenticatedClient(BACKUP_SERVICE).listBackups(null));
+    assertEquals(OK, getAuthenticatedClient(BACKUP_SERVICE).listBackups());
   }
 
   private HttpResponse callH2Console(Credentials credentials) throws Exception {

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/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 650f214..1d6af3d 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
@@ -24,7 +24,6 @@ import org.apache.aurora.common.testing.easymock.EasyMockTest;
 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;
@@ -55,7 +54,7 @@ public class ShiroAuthorizingInterceptorTest extends EasyMockTest {
     subject = createMock(Subject.class);
     statsProvider = createMock(StatsProvider.class);
     methodInvocation = createMock(MethodInvocation.class);
-    interceptedMethod = AuroraAdmin.Iface.class.getMethod("snapshot", SessionKey.class);
+    interceptedMethod = AuroraAdmin.Iface.class.getMethod("snapshot");
     expect(statsProvider.makeCounter(SHIRO_AUTHORIZATION_FAILURES)).andReturn(new AtomicLong());
   }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/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 d03cd2a..16a3a3b 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
@@ -111,12 +111,12 @@ public class ShiroAuthorizingParamInterceptorTest extends EasyMockTest {
     expect(subject
         .isPermitted(interceptor.makeTargetPermission("createJob", JOB_KEY)))
         .andReturn(true);
-    expect(thrift.createJob(jobConfiguration, null, null))
+    expect(thrift.createJob(jobConfiguration, null))
         .andReturn(response);
 
     replayAndInitialize();
 
-    assertSame(response, decoratedThrift.createJob(jobConfiguration, null, null));
+    assertSame(response, decoratedThrift.createJob(jobConfiguration, null));
   }
 
   @Test
@@ -126,12 +126,12 @@ public class ShiroAuthorizingParamInterceptorTest extends EasyMockTest {
 
     expect(subject.isPermitted(interceptor.makeWildcardPermission("killTasks")))
         .andReturn(true);
-    expect(thrift.killTasks(taskQuery, null, null))
+    expect(thrift.killTasks(taskQuery, null))
         .andReturn(response);
 
     replayAndInitialize();
 
-    assertSame(response, decoratedThrift.killTasks(taskQuery, null, null));
+    assertSame(response, decoratedThrift.killTasks(taskQuery, null));
   }
 
   @Test
@@ -145,7 +145,7 @@ public class ShiroAuthorizingParamInterceptorTest extends EasyMockTest {
 
     assertEquals(
         ResponseCode.INVALID_REQUEST,
-        decoratedThrift.killTasks(taskQuery, null, null).getResponseCode());
+        decoratedThrift.killTasks(taskQuery, null).getResponseCode());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
index 2bfc2a7..1fd106b 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -68,7 +68,6 @@ import org.apache.aurora.gen.Result;
 import org.apache.aurora.gen.RewriteConfigsRequest;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.gen.ScheduledTask;
-import org.apache.aurora.gen.SessionKey;
 import org.apache.aurora.gen.StartJobUpdateResult;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskConstraint;
@@ -164,7 +163,6 @@ import static org.junit.Assert.fail;
 
 public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
-  private static final SessionKey SESSION = null;
   private static final String AUDIT_MESSAGE = "message";
   private static final AuditData AUDIT = new AuditData(USER, Optional.of(AUDIT_MESSAGE));
 
@@ -262,7 +260,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.createJob(jobConfig, null, SESSION));
+    assertOkResponse(thrift.createJob(jobConfig, null));
   }
 
   @Test
@@ -283,7 +281,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.createJob(job.newBuilder(), LOCK.newBuilder(), SESSION));
+    assertOkResponse(thrift.createJob(job.newBuilder(), LOCK.newBuilder()));
   }
 
   @Test
@@ -294,7 +292,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(NO_CRON),
-        thrift.createJob(job.newBuilder(), LOCK.newBuilder(), SESSION));
+        thrift.createJob(job.newBuilder(), LOCK.newBuilder()));
   }
 
   @Test
@@ -304,7 +302,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertResponse(
         INVALID_REQUEST,
-        thrift.createJob(job.newBuilder(), null, SESSION));
+        thrift.createJob(job.newBuilder(), null));
   }
 
   @Test
@@ -315,7 +313,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(LOCK_ERROR, thrift.createJob(job.newBuilder(), LOCK.newBuilder(), SESSION));
+    assertResponse(LOCK_ERROR, thrift.createJob(job.newBuilder(), LOCK.newBuilder()));
   }
 
   @Test
@@ -326,7 +324,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null));
   }
 
   @Test
@@ -338,7 +336,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null));
   }
 
   @Test
@@ -356,7 +354,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null));
   }
 
   @Test
@@ -376,7 +374,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null));
   }
 
   @Test
@@ -392,7 +390,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(job.newBuilder(), null));
   }
 
   private void assertMessageMatches(Response response, final String string) {
@@ -407,7 +405,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     JobConfiguration job =
         new JobConfiguration().setKey(JOB_KEY.newBuilder()).setOwner(ROLE_IDENTITY);
-    assertResponse(INVALID_REQUEST, thrift.createJob(job, null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(job, null));
   }
 
   @Test
@@ -417,7 +415,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    Response response = thrift.createJob(job, LOCK.newBuilder(), SESSION);
+    Response response = thrift.createJob(job, LOCK.newBuilder());
     assertResponse(INVALID_REQUEST, response);
     // TODO(wfarner): Don't rely on a magic string here, reference a constant from the source.
     assertMessageMatches(response, "Configuration may not be null");
@@ -430,7 +428,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.createJob(job, null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(job, null));
   }
 
   @Test
@@ -440,7 +438,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.createJob(job, null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(job, null));
   }
 
   @Test
@@ -451,7 +449,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     task.setNumCpus(0);
     task.setRamMb(0);
     task.setDiskMb(0);
-    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null));
   }
 
   @Test
@@ -459,7 +457,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
 
     TaskConfig task = productionTask().setNumCpus(0.0);
-    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null));
   }
 
   @Test
@@ -467,7 +465,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
 
     TaskConfig task = productionTask().setRamMb(-123);
-    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null));
   }
 
   @Test
@@ -475,7 +473,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
 
     TaskConfig task = productionTask().setDiskMb(0);
-    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null));
   }
 
   @Test
@@ -522,7 +520,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.createJob(job, null, SESSION));
+    assertOkResponse(thrift.createJob(job, null));
   }
 
   @Test
@@ -531,7 +529,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     TaskConfig task = nonProductionTask();
     task.setConstraints(ImmutableSet.of(dedicatedConstraint(ImmutableSet.of("mesos"))));
-    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null));
   }
 
   @Test
@@ -540,7 +538,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     TaskConfig task = nonProductionTask();
     task.setConstraints(ImmutableSet.of(dedicatedConstraint(1)));
-    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null));
   }
 
   @Test
@@ -549,7 +547,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     TaskConfig task = nonProductionTask();
     task.setConstraints(ImmutableSet.of(dedicatedConstraint(ImmutableSet.of("mesos", "test"))));
-    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null));
   }
 
   private IScheduledTask buildTaskForJobUpdate(int instanceId) {
@@ -600,7 +598,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertEquals(okEmptyResponse(), thrift.killTasks(query, null, SESSION));
+    assertEquals(okEmptyResponse(), thrift.killTasks(query, null));
   }
 
   @Test
@@ -612,7 +610,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.killTasks(query.get(), null, SESSION));
+    assertOkResponse(thrift.killTasks(query.get(), null));
   }
 
   @Test
@@ -628,7 +626,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(LOCK_ERROR, thrift.killTasks(query.get(), LOCK.newBuilder(), SESSION));
+    assertResponse(LOCK_ERROR, thrift.killTasks(query.get(), LOCK.newBuilder()));
   }
 
   @Test
@@ -643,7 +641,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.killTasks(query.get(), null, SESSION));
+    assertOkResponse(thrift.killTasks(query.get(), null));
   }
 
   @Test
@@ -654,7 +652,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.killTasks(query, null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.killTasks(query, null));
   }
 
   @Test
@@ -664,7 +662,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    Response response = thrift.killTasks(query.get(), null, SESSION);
+    Response response = thrift.killTasks(query.get(), null);
     assertOkResponse(response);
     assertMessageMatches(response, SchedulerThriftInterface.NO_TASKS_TO_KILL_MESSAGE);
   }
@@ -682,7 +680,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.setQuota(ROLE, resourceAggregate, SESSION));
+    assertOkResponse(thrift.setQuota(ROLE, resourceAggregate));
   }
 
   @Test
@@ -700,7 +698,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.setQuota(ROLE, resourceAggregate, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.setQuota(ROLE, resourceAggregate));
   }
 
   @Test
@@ -717,7 +715,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.forceTaskState(TASK_ID, status, SESSION));
+    assertOkResponse(thrift.forceTaskState(TASK_ID, status));
   }
 
   @Test
@@ -743,24 +741,24 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertEquals(okEmptyResponse(), thrift.performBackup(SESSION));
+    assertEquals(okEmptyResponse(), thrift.performBackup());
 
     assertEquals(
         okResponse(Result.listBackupsResult(new ListBackupsResult().setBackups(backups))),
-        thrift.listBackups(SESSION));
+        thrift.listBackups());
 
-    assertEquals(okEmptyResponse(), thrift.stageRecovery(backupId, SESSION));
+    assertEquals(okEmptyResponse(), thrift.stageRecovery(backupId));
 
     assertEquals(
         okResponse(Result.queryRecoveryResult(
             new QueryRecoveryResult().setTasks(IScheduledTask.toBuildersSet(queryResult)))),
-        thrift.queryRecovery(query.get(), SESSION));
+        thrift.queryRecovery(query.get()));
 
-    assertEquals(okEmptyResponse(), thrift.deleteRecoveryTasks(query.get(), SESSION));
+    assertEquals(okEmptyResponse(), thrift.deleteRecoveryTasks(query.get()));
 
-    assertEquals(okEmptyResponse(), thrift.commitRecovery(SESSION));
+    assertEquals(okEmptyResponse(), thrift.commitRecovery());
 
-    assertEquals(okEmptyResponse(), thrift.unloadRecovery(SESSION));
+    assertEquals(okEmptyResponse(), thrift.unloadRecovery());
   }
 
   @Test
@@ -774,7 +772,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
 
     try {
-      thrift.stageRecovery(backupId, SESSION);
+      thrift.stageRecovery(backupId);
       fail("No recovery exception thrown.");
     } catch (RecoveryException e) {
       assertEquals(recoveryException.getMessage(), e.getMessage());
@@ -802,7 +800,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
 
     assertOkResponse(
-        thrift.restartShards(JOB_KEY.newBuilder(), shards, LOCK.newBuilder(), SESSION));
+        thrift.restartShards(JOB_KEY.newBuilder(), shards, LOCK.newBuilder()));
   }
 
   @Test
@@ -816,7 +814,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertResponse(
         LOCK_ERROR,
-        thrift.restartShards(JOB_KEY.newBuilder(), shards, LOCK.newBuilder(), SESSION));
+        thrift.restartShards(JOB_KEY.newBuilder(), shards, LOCK.newBuilder()));
   }
 
   @Test
@@ -830,7 +828,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertResponse(
         INVALID_REQUEST,
-        thrift.restartShards(JOB_KEY.newBuilder(), shards, LOCK.newBuilder(), SESSION));
+        thrift.restartShards(JOB_KEY.newBuilder(), shards, LOCK.newBuilder()));
   }
 
   @Test
@@ -848,7 +846,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     // Validate key is populated during sanitizing.
     JobConfiguration jobConfig = CRON_JOB;
     jobConfig.getTaskConfig().unsetJob();
-    assertOkResponse(thrift.replaceCronTemplate(jobConfig, null, SESSION));
+    assertOkResponse(thrift.replaceCronTemplate(jobConfig, null));
   }
 
   @Test
@@ -857,7 +855,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     expectLastCall().andThrow(new LockException("Failed lock."));
     control.replay();
 
-    assertResponse(LOCK_ERROR, thrift.replaceCronTemplate(CRON_JOB, null, SESSION));
+    assertResponse(LOCK_ERROR, thrift.replaceCronTemplate(CRON_JOB, null));
   }
 
   @Test
@@ -874,14 +872,14 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.replaceCronTemplate(CRON_JOB, null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.replaceCronTemplate(CRON_JOB, null));
   }
 
   @Test
   public void testStartCronJob() throws Exception {
     cronJobManager.startJobNow(JOB_KEY);
     control.replay();
-    assertResponse(OK, thrift.startCronJob(JOB_KEY.newBuilder(), SESSION));
+    assertResponse(OK, thrift.startCronJob(JOB_KEY.newBuilder()));
   }
 
   @Test
@@ -889,7 +887,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     cronJobManager.startJobNow(JOB_KEY);
     expectLastCall().andThrow(new CronException("failed"));
     control.replay();
-    assertResponse(INVALID_REQUEST, thrift.startCronJob(JOB_KEY.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startCronJob(JOB_KEY.newBuilder()));
   }
 
   @Test
@@ -906,7 +904,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     storageUtil.expectTaskFetch(Query.jobScoped(JOB_KEY).active());
     cronJobManager.createJob(SanitizedCronJob.from(sanitized));
     control.replay();
-    assertResponse(OK, thrift.scheduleCronJob(CRON_JOB, null, SESSION));
+    assertResponse(OK, thrift.scheduleCronJob(CRON_JOB, null));
   }
 
   @Test
@@ -924,7 +922,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
     assertEquals(
         invalidResponse(jobAlreadyExistsMessage(JOB_KEY)),
-        thrift.scheduleCronJob(CRON_JOB, null, SESSION));
+        thrift.scheduleCronJob(CRON_JOB, null));
   }
 
   @Test
@@ -944,7 +942,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     // Validate key is populated during sanitizing.
     JobConfiguration jobConfig = CRON_JOB;
     jobConfig.getTaskConfig().unsetJob();
-    assertResponse(OK, thrift.scheduleCronJob(jobConfig, null, SESSION));
+    assertResponse(OK, thrift.scheduleCronJob(jobConfig, null));
   }
 
   @Test
@@ -953,7 +951,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     IJobConfiguration job = IJobConfiguration.build(makeJob(null));
     assertResponse(
         INVALID_REQUEST,
-        thrift.scheduleCronJob(job.newBuilder(), null, SESSION));
+        thrift.scheduleCronJob(job.newBuilder(), null));
   }
 
   @Test
@@ -961,7 +959,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     lockManager.validateIfLocked(LOCK_KEY, java.util.Optional.of(LOCK));
     expectLastCall().andThrow(new LockException("Failed lock"));
     control.replay();
-    assertResponse(LOCK_ERROR, thrift.scheduleCronJob(CRON_JOB, LOCK.newBuilder(), SESSION));
+    assertResponse(LOCK_ERROR, thrift.scheduleCronJob(CRON_JOB, LOCK.newBuilder()));
   }
 
   @Test
@@ -970,7 +968,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(noCronScheduleMessage(JOB_KEY)),
-        thrift.scheduleCronJob(makeJob(), null, SESSION));
+        thrift.scheduleCronJob(makeJob(), null));
   }
 
   @Test
@@ -984,7 +982,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     expectCronQuotaCheck(sanitized.getJobConfig(), NOT_ENOUGH_QUOTA);
 
     control.replay();
-    assertResponse(INVALID_REQUEST, thrift.scheduleCronJob(CRON_JOB, null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.scheduleCronJob(CRON_JOB, null));
   }
 
   @Test
@@ -994,7 +992,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(OK, thrift.descheduleCronJob(CRON_JOB.getKey(), null, SESSION));
+    assertResponse(OK, thrift.descheduleCronJob(CRON_JOB.getKey(), null));
   }
 
   @Test
@@ -1002,7 +1000,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     lockManager.validateIfLocked(LOCK_KEY, java.util.Optional.empty());
     expectLastCall().andThrow(new LockException("Failed lock"));
     control.replay();
-    assertResponse(LOCK_ERROR, thrift.descheduleCronJob(CRON_JOB.getKey(), null, SESSION));
+    assertResponse(LOCK_ERROR, thrift.descheduleCronJob(CRON_JOB.getKey(), null));
   }
 
   @Test
@@ -1013,7 +1011,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(notScheduledCronMessage(JOB_KEY)),
-        thrift.descheduleCronJob(JOB_KEY.newBuilder(), null, SESSION));
+        thrift.descheduleCronJob(JOB_KEY.newBuilder(), null));
   }
 
   @Test
@@ -1029,7 +1027,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     RewriteConfigsRequest request = new RewriteConfigsRequest(
         ImmutableList.of(ConfigRewrite.instanceRewrite(
             new InstanceConfigRewrite(instance, productionTask(), productionTask()))));
-    assertResponse(WARNING, thrift.rewriteConfigs(request, SESSION));
+    assertResponse(WARNING, thrift.rewriteConfigs(request));
   }
 
   @Test
@@ -1037,7 +1035,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
 
     RewriteConfigsRequest request = new RewriteConfigsRequest(ImmutableList.of());
-    assertResponse(INVALID_REQUEST, thrift.rewriteConfigs(request, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.rewriteConfigs(request));
   }
 
   @Test(expected = RuntimeException.class)
@@ -1048,8 +1046,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     thrift.rewriteConfigs(
         new RewriteConfigsRequest(
             ImmutableList.of(ConfigRewrite.jobRewrite(
-                new JobConfigRewrite(job.newBuilder(), job.newBuilder().setTaskConfig(null))))),
-        SESSION);
+                new JobConfigRewrite(job.newBuilder(), job.newBuilder().setTaskConfig(null))))));
   }
 
   @Test
@@ -1067,7 +1064,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
                     .setOwner(rewrittenIdentity))
                 .setOwner(rewrittenIdentity)
                 .setKey(rewrittenJobKey)))));
-    assertResponse(WARNING, thrift.rewriteConfigs(request, SESSION));
+    assertResponse(WARNING, thrift.rewriteConfigs(request));
   }
 
   @Test
@@ -1092,7 +1089,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     RewriteConfigsRequest request = new RewriteConfigsRequest(
         ImmutableList.of(ConfigRewrite.instanceRewrite(
             new InstanceConfigRewrite(instance, modifiedConfig, modifiedConfig))));
-    assertResponse(WARNING, thrift.rewriteConfigs(request, SESSION));
+    assertResponse(WARNING, thrift.rewriteConfigs(request));
   }
 
   @Test
@@ -1121,7 +1118,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     RewriteConfigsRequest request = new RewriteConfigsRequest(
         ImmutableList.of(ConfigRewrite.instanceRewrite(
             new InstanceConfigRewrite(instanceKey, storedConfig, modifiedConfig.newBuilder()))));
-    assertOkResponse(thrift.rewriteConfigs(request, SESSION));
+    assertOkResponse(thrift.rewriteConfigs(request));
   }
 
   @Test
@@ -1151,7 +1148,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     RewriteConfigsRequest request = new RewriteConfigsRequest(
         ImmutableList.of(ConfigRewrite.instanceRewrite(
             new InstanceConfigRewrite(instanceKey, config, config))));
-    assertResponse(WARNING, thrift.rewriteConfigs(request, SESSION));
+    assertResponse(WARNING, thrift.rewriteConfigs(request));
   }
 
   @Test
@@ -1167,7 +1164,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     RewriteConfigsRequest request = new RewriteConfigsRequest(
         ImmutableList.of(ConfigRewrite.jobRewrite(
             new JobConfigRewrite(newJob, newJob))));
-    assertResponse(WARNING, thrift.rewriteConfigs(request, SESSION));
+    assertResponse(WARNING, thrift.rewriteConfigs(request));
   }
 
   @Test
@@ -1183,7 +1180,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     RewriteConfigsRequest request = new RewriteConfigsRequest(
         ImmutableList.of(ConfigRewrite.jobRewrite(
             new JobConfigRewrite(oldJob, newJob))));
-    assertResponse(WARNING, thrift.rewriteConfigs(request, SESSION));
+    assertResponse(WARNING, thrift.rewriteConfigs(request));
   }
 
   @Test
@@ -1205,7 +1202,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     RewriteConfigsRequest request = new RewriteConfigsRequest(
         ImmutableList.of(ConfigRewrite.jobRewrite(
             new JobConfigRewrite(oldJob, newJob))));
-    assertOkResponse(thrift.rewriteConfigs(request, SESSION));
+    assertOkResponse(thrift.rewriteConfigs(request));
   }
 
   @Test
@@ -1214,7 +1211,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     TaskConfig task = nonProductionTask();
     task.setConstraints(ImmutableSet.of(dedicatedConstraint(ImmutableSet.of("mesos"))));
-    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.createJob(makeJob(task), null));
   }
 
   @Test
@@ -1237,26 +1234,26 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         none,
-        thrift.maintenanceStatus(hosts, SESSION).getResult().getMaintenanceStatusResult()
+        thrift.maintenanceStatus(hosts).getResult().getMaintenanceStatusResult()
             .getStatuses());
     assertEquals(
         scheduled,
-        thrift.startMaintenance(hosts, SESSION).getResult().getStartMaintenanceResult()
+        thrift.startMaintenance(hosts).getResult().getStartMaintenanceResult()
             .getStatuses());
     assertEquals(
         draining,
-        thrift.drainHosts(hosts, SESSION).getResult().getDrainHostsResult().getStatuses());
+        thrift.drainHosts(hosts).getResult().getDrainHostsResult().getStatuses());
     assertEquals(
         draining,
-        thrift.maintenanceStatus(hosts, SESSION).getResult().getMaintenanceStatusResult()
+        thrift.maintenanceStatus(hosts).getResult().getMaintenanceStatusResult()
             .getStatuses());
     assertEquals(
         drained,
-        thrift.maintenanceStatus(hosts, SESSION).getResult().getMaintenanceStatusResult()
+        thrift.maintenanceStatus(hosts).getResult().getMaintenanceStatusResult()
             .getStatuses());
     assertEquals(
         none,
-        thrift.endMaintenance(hosts, SESSION).getResult().getEndMaintenanceResult().getStatuses());
+        thrift.endMaintenance(hosts).getResult().getEndMaintenanceResult().getStatuses());
   }
 
   private static Response okEmptyResponse() {
@@ -1279,10 +1276,10 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.snapshot(SESSION));
+    assertOkResponse(thrift.snapshot());
 
     try {
-      thrift.snapshot(SESSION);
+      thrift.snapshot();
       fail("No StorageException thrown.");
     } catch (StorageException e) {
       // Expected.
@@ -1315,7 +1312,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     // Validate key is populated during sanitizing.
     AddInstancesConfig config = createInstanceConfig(populatedTask.newBuilder());
     config.getTaskConfig().unsetJob();
-    assertOkResponse(thrift.addInstances(config, LOCK.newBuilder(), SESSION));
+    assertOkResponse(thrift.addInstances(config, LOCK.newBuilder()));
   }
 
   @Test
@@ -1335,7 +1332,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.addInstances(config, null, SESSION));
+    assertOkResponse(thrift.addInstances(config, null));
   }
 
   @Test
@@ -1344,7 +1341,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder()));
   }
 
   @Test
@@ -1354,7 +1351,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder()));
   }
 
   @Test(expected = StorageException.class)
@@ -1364,7 +1361,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    thrift.addInstances(config, LOCK.newBuilder(), SESSION);
+    thrift.addInstances(config, LOCK.newBuilder());
   }
 
   @Test
@@ -1376,7 +1373,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(LOCK_ERROR, thrift.addInstances(config, LOCK.newBuilder(), SESSION));
+    assertResponse(LOCK_ERROR, thrift.addInstances(config, LOCK.newBuilder()));
   }
 
   @Test
@@ -1395,7 +1392,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder()));
   }
 
   @Test
@@ -1411,7 +1408,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder()));
   }
 
   @Test
@@ -1432,7 +1429,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.addInstances(config, LOCK.newBuilder()));
   }
 
   @Test
@@ -1442,7 +1439,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    Response response = thrift.acquireLock(LOCK_KEY.newBuilder(), SESSION);
+    Response response = thrift.acquireLock(LOCK_KEY.newBuilder());
     assertEquals(LOCK.newBuilder(), response.getResult().getAcquireLockResult().getLock());
   }
 
@@ -1454,7 +1451,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(LOCK_ERROR, thrift.acquireLock(LOCK_KEY.newBuilder(), SESSION));
+    assertResponse(LOCK_ERROR, thrift.acquireLock(LOCK_KEY.newBuilder()));
   }
 
   @Test
@@ -1464,7 +1461,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.releaseLock(LOCK.newBuilder(), CHECKED, SESSION));
+    assertOkResponse(thrift.releaseLock(LOCK.newBuilder(), CHECKED));
   }
 
   @Test
@@ -1474,7 +1471,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(LOCK_ERROR, thrift.releaseLock(LOCK.newBuilder(), CHECKED, SESSION));
+    assertResponse(LOCK_ERROR, thrift.releaseLock(LOCK.newBuilder(), CHECKED));
   }
 
   @Test
@@ -1483,7 +1480,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertEquals(okEmptyResponse(), thrift.releaseLock(LOCK.newBuilder(), UNCHECKED, SESSION));
+    assertEquals(okEmptyResponse(), thrift.releaseLock(LOCK.newBuilder(), UNCHECKED));
   }
 
   @Test
@@ -1530,7 +1527,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     JobUpdateRequest request = buildJobUpdateRequest(update);
     request.getTaskConfig().unsetJob();
 
-    Response response = assertOkResponse(thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    Response response = assertOkResponse(thrift.startJobUpdate(request, AUDIT_MESSAGE));
     assertEquals(
         new StartJobUpdateResult(UPDATE_KEY.newBuilder()),
         response.getResult().getStartJobUpdateResult());
@@ -1578,7 +1575,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
 
     Response response = assertOkResponse(
-        thrift.startJobUpdate(buildJobUpdateRequest(update), AUDIT_MESSAGE, SESSION));
+        thrift.startJobUpdate(buildJobUpdateRequest(update), AUDIT_MESSAGE));
     assertEquals(
         new StartJobUpdateResult(UPDATE_KEY.newBuilder()),
         response.getResult().getStartJobUpdateResult());
@@ -1587,7 +1584,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
   @Test(expected = NullPointerException.class)
   public void testStartUpdateFailsNullRequest() throws Exception {
     control.replay();
-    thrift.startJobUpdate(null, AUDIT_MESSAGE, SESSION);
+    thrift.startJobUpdate(null, AUDIT_MESSAGE);
   }
 
   @Test(expected = NullPointerException.class)
@@ -1595,8 +1592,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
     thrift.startJobUpdate(
         new JobUpdateRequest(null, 5, buildJobUpdateSettings()),
-        AUDIT_MESSAGE,
-        SESSION);
+        AUDIT_MESSAGE);
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -1610,8 +1606,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
                 .setOwner(new Identity(ROLE, null)),
             5,
             buildJobUpdateSettings()),
-        AUDIT_MESSAGE,
-        SESSION);
+        AUDIT_MESSAGE);
   }
 
   @Test
@@ -1623,7 +1618,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(SchedulerThriftInterface.INVALID_GROUP_SIZE),
-        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE, SESSION));
+        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1635,7 +1630,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(SchedulerThriftInterface.INVALID_MAX_INSTANCE_FAILURES),
-        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE, SESSION));
+        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1649,7 +1644,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(SchedulerThriftInterface.TOO_MANY_POTENTIAL_FAILED_INSTANCES),
-        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE, SESSION));
+        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1661,7 +1656,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(SchedulerThriftInterface.INVALID_MAX_FAILED_INSTANCES),
-        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE, SESSION));
+        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1673,14 +1668,14 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(SchedulerThriftInterface.INVALID_MIN_WAIT_TO_RUNNING),
-        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE, SESSION));
+        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE));
   }
 
   @Test
   public void testStartUpdateFailsNonServiceTask() throws Exception {
     control.replay();
     JobUpdateRequest request = buildJobUpdateRequest(populatedTask().setIsService(false));
-    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1692,7 +1687,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         invalidResponse(SchedulerThriftInterface.INVALID_PULSE_TIMEOUT),
-        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE, SESSION));
+        thrift.startJobUpdate(updateRequest, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1701,7 +1696,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     expectCronJob();
 
     control.replay();
-    assertEquals(invalidResponse(NO_CRON), thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    assertEquals(invalidResponse(NO_CRON), thrift.startJobUpdate(request, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1710,7 +1705,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
         buildJobUpdateRequest(populatedTask().setIsService(true).setNumCpus(-1));
 
     control.replay();
-    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1729,7 +1724,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
     JobUpdateRequest request = buildJobUpdateRequest(update);
-    Response response = thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION);
+    Response response = thrift.startJobUpdate(request, AUDIT_MESSAGE);
     assertResponse(OK, response);
     assertEquals(
         NOOP_JOB_UPDATE_MESSAGE,
@@ -1755,7 +1750,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
     JobUpdateRequest request = buildJobUpdateRequest(IJobUpdate.build(builder));
-    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1794,7 +1789,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
     JobUpdateRequest request = buildJobUpdateRequest(IJobUpdate.build(builder));
-    assertResponse(OK, thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    assertResponse(OK, thrift.startJobUpdate(request, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1809,7 +1804,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1826,7 +1821,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1845,7 +1840,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE, SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, AUDIT_MESSAGE));
   }
 
   @Test
@@ -1873,7 +1868,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertResponse(
         INVALID_REQUEST,
-        thrift.startJobUpdate(buildJobUpdateRequest(update), AUDIT_MESSAGE, SESSION));
+        thrift.startJobUpdate(buildJobUpdateRequest(update), AUDIT_MESSAGE));
   }
 
   @Test
@@ -1883,7 +1878,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(OK, thrift.pauseJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE, SESSION));
+    assertResponse(OK, thrift.pauseJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE));
   }
 
   @Test
@@ -1893,7 +1888,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(OK, thrift.pauseJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE, SESSION));
+    assertResponse(OK, thrift.pauseJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE));
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -1906,8 +1901,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
         OK,
         thrift.pauseJobUpdate(
             UPDATE_KEY.newBuilder(),
-            Strings.repeat("*", AuditData.MAX_MESSAGE_LENGTH + 1),
-            SESSION));
+            Strings.repeat("*", AuditData.MAX_MESSAGE_LENGTH + 1)));
   }
 
   @Test
@@ -1920,7 +1914,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertResponse(
         INVALID_REQUEST,
-        thrift.pauseJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE, SESSION));
+        thrift.pauseJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE));
   }
 
   @Test
@@ -1930,7 +1924,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(OK, thrift.resumeJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE, SESSION));
+    assertResponse(OK, thrift.resumeJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE));
   }
 
   @Test
@@ -1943,7 +1937,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertResponse(
         INVALID_REQUEST,
-        thrift.resumeJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE, SESSION));
+        thrift.resumeJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE));
   }
 
   @Test
@@ -1953,7 +1947,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(OK, thrift.abortJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE, SESSION));
+    assertResponse(OK, thrift.abortJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE));
   }
 
   @Test
@@ -1963,7 +1957,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(OK, thrift.abortJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE, SESSION));
+    assertResponse(OK, thrift.abortJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE));
   }
 
   @Test
@@ -1976,7 +1970,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertResponse(
         INVALID_REQUEST,
-        thrift.abortJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE, SESSION));
+        thrift.abortJobUpdate(UPDATE_KEY.newBuilder(), AUDIT_MESSAGE));
   }
 
   @Test
@@ -1988,7 +1982,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     assertEquals(
         okResponse(Result.pulseJobUpdateResult(
             new PulseJobUpdateResult(JobUpdatePulseStatus.OK))),
-        thrift.pulseJobUpdate(UPDATE_KEY.newBuilder(), SESSION));
+        thrift.pulseJobUpdate(UPDATE_KEY.newBuilder()));
   }
 
   @Test
@@ -1999,7 +1993,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertEquals(
         okResponse(Result.pulseJobUpdateResult(new PulseJobUpdateResult(JobUpdatePulseStatus.OK))),
-        thrift.pulseJobUpdate(UPDATE_KEY.newBuilder(), SESSION));
+        thrift.pulseJobUpdate(UPDATE_KEY.newBuilder()));
   }
 
   @Test
@@ -2008,7 +2002,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.pulseJobUpdate(UPDATE_KEY.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.pulseJobUpdate(UPDATE_KEY.newBuilder()));
   }
 
   @Test
@@ -2024,8 +2018,6 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     assertEquals(updateSummary, thrift.getJobUpdateSummaries(new JobUpdateQuery()));
   }
 
-  private static final String AUTH_DENIED_MESSAGE = "Denied!";
-
   private IExpectationSetters<String> expectGetRemoteUser() {
     return expect(auditMessages.getRemoteUserName()).andReturn(USER);
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java b/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
index 63c20d1..72b5c30 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
@@ -116,6 +116,6 @@ public class ThriftIT extends EasyMockTest {
 
     assertEquals(
         OK,
-        thrift.setQuota(USER, QUOTA.newBuilder(), null).getResponseCode());
+        thrift.setQuota(USER, QUOTA.newBuilder()).getResponseCode());
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/7b1c2b2f/src/test/java/org/apache/aurora/scheduler/thrift/aop/AopModuleTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/aop/AopModuleTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/aop/AopModuleTest.java
index dafcf25..19b7415 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/aop/AopModuleTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/aop/AopModuleTest.java
@@ -27,7 +27,6 @@ import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.Response;
 import org.apache.aurora.gen.ResponseCode;
 import org.apache.aurora.gen.ServerInfo;
-import org.apache.aurora.gen.SessionKey;
 import org.apache.aurora.scheduler.storage.entities.IServerInfo;
 import org.junit.Before;
 import org.junit.Test;
@@ -76,12 +75,11 @@ public class AopModuleTest extends EasyMockTest {
   @Test
   public void testFlaggedMethodDisabled() throws Exception {
     JobConfiguration job = new JobConfiguration();
-    SessionKey session = new SessionKey();
 
     control.replay();
 
     Iface thrift = getIface(ImmutableMap.of("createJob", false));
-    assertEquals(ResponseCode.ERROR, thrift.createJob(job, null, session).getResponseCode());
+    assertEquals(ResponseCode.ERROR, thrift.createJob(job, null).getResponseCode());
   }
 
   @Test(expected = CreationException.class)
@@ -93,12 +91,12 @@ public class AopModuleTest extends EasyMockTest {
   private void assertCreateAllowed(Map<String, Boolean> toggledMethods) throws Exception {
     JobConfiguration job = new JobConfiguration();
     Response response = new Response();
-    expect(mockThrift.createJob(job, null, null)).andReturn(response);
+    expect(mockThrift.createJob(job, null)).andReturn(response);
 
     control.replay();
 
     Iface thrift = getIface(toggledMethods);
-    assertSame(response, thrift.createJob(job, null, null));
+    assertSame(response, thrift.createJob(job, null));
   }
 
   @Test