You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2014/08/18 23:03:39 UTC

git commit: Dropping lock parameter from update APIs.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 4f6093bb2 -> f08c648b4


Dropping lock parameter from update APIs.

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


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

Branch: refs/heads/master
Commit: f08c648b45961a8c22b07d03258f557e5a40f917
Parents: 4f6093b
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Mon Aug 18 14:03:19 2014 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Mon Aug 18 14:03:19 2014 -0700

----------------------------------------------------------------------
 .../thrift/SchedulerThriftInterface.java        |  9 ++++-----
 .../python/apache/aurora/client/api/__init__.py | 11 +++++++---
 .../thrift/org/apache/aurora/gen/api.thrift     | 21 ++++++++++----------
 .../thrift/SchedulerThriftInterfaceTest.java    | 10 +++++-----
 .../scheduler/thrift/aop/ForwardingThrift.java  | 16 +++++++--------
 .../python/apache/aurora/client/api/test_api.py |  7 +++----
 .../aurora/client/api/test_scheduler_client.py  | 16 +++++++--------
 .../org/apache/aurora/gen/api.thrift.md5        |  2 +-
 8 files changed, 48 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f08c648b/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 7ef2885..0d8d75b 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -1258,7 +1258,6 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
   @Override
   public Response startJobUpdate(
       JobUpdateRequest mutableRequest,
-      Lock mutableLock,
       SessionKey session) {
 
     // TODO(maxim): validate JobUpdateRequest fields.
@@ -1288,7 +1287,7 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
       @Override
       public Response apply(MutableStoreProvider storeProvider) {
         // TODO(wfarner): Move lock acquisition down into the update controller once introduced.
-        ILock lock;
+        final ILock lock;
         try {
           lock = lockManager.acquireLock(
               ILockKey.build(LockKey.job(request.getJobKey().newBuilder())),
@@ -1311,17 +1310,17 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
   }
 
   @Override
-  public Response pauseJobUpdate(String updateId, Lock lock, SessionKey session) {
+  public Response pauseJobUpdate(JobKey jobKey, SessionKey session) {
     throw new UnsupportedOperationException("Not implemented");
   }
 
   @Override
-  public Response resumeJobUpdate(String updateId, Lock lock, SessionKey session) {
+  public Response resumeJobUpdate(JobKey jobKey, SessionKey session) {
     throw new UnsupportedOperationException("Not implemented");
   }
 
   @Override
-  public Response abortJobUpdate(String updateId, Lock lock, SessionKey session) {
+  public Response abortJobUpdate(JobKey jobKey, SessionKey session) {
     throw new UnsupportedOperationException("Not implemented");
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f08c648b/src/main/python/apache/aurora/client/api/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/__init__.py b/src/main/python/apache/aurora/client/api/__init__.py
index 62de93b..4968738 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -149,6 +149,8 @@ class AuroraClientAPI(object):
 
     Arguments:
     job_key - AuroraJobKey instance.
+
+    Returns response object with acquired job lock.
     """
     self._assert_valid_job_key(job_key)
     return self._scheduler_proxy.acquireLock(LockKey(job=job_key.to_thrift()))
@@ -160,17 +162,20 @@ class AuroraClientAPI(object):
 
     Arguments:
     lock - Previously acquired Lock instance.
+
+    Returns response object.
     """
     self._assert_valid_lock(lock)
     return self._scheduler_proxy.releaseLock(lock, LockValidation.CHECKED)
 
-  def start_job_update(self, config, lock, instances=None):
+  def start_job_update(self, config, instances=None):
     """Requests Scheduler to start job update process.
 
     Arguments:
     config - AuroraConfig instance with update details.
-    lock - Job Lock instance to ensure exclusive job mutation access.
     instances - Optional list of instances to restrict update to.
+
+    Returns response object with update ID and acquired job lock.
     """
     try:
       settings = UpdaterConfig(**config.update_config().get()).to_thrift_update_settings(instances)
@@ -184,7 +189,7 @@ class AuroraClientAPI(object):
         taskConfig=config.job().taskConfig
     )
 
-    return self._scheduler_proxy.startJobUpdate(request, lock)
+    return self._scheduler_proxy.startJobUpdate(request)
 
   def cancel_update(self, job_key):
     """Cancel the update represented by job_key. Returns whether or not the cancellation was

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f08c648b/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/src/main/thrift/org/apache/aurora/gen/api.thrift b/src/main/thrift/org/apache/aurora/gen/api.thrift
index 2b376d6..a78a4d8 100644
--- a/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -768,6 +768,7 @@ struct GetPendingReasonResult {
 
 /** Result of the startUpdate call. */
 struct StartJobUpdateResult {
+  /** Job update ID. */
   1: string updateId
 }
 
@@ -947,19 +948,19 @@ service AuroraSchedulerManager extends ReadOnlyScheduler {
   Response replaceCronTemplate(1: JobConfiguration config, 2: Lock lock, 3: SessionKey session)
 
   /** Starts update of the existing service job. Not implemented yet. */
-  Response startJobUpdate(
-      1: JobUpdateRequest request,
-      2: Lock lock,
-      3: SessionKey session)
+  Response startJobUpdate(1: JobUpdateRequest request, 2: SessionKey session)
 
-  /** Pauses the update progress. Can be resumed by resumeUpdate call. Not implemented yet. */
-  Response pauseJobUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
+  /**
+   * Pauses the update progress for the specified job. Can be resumed by resumeUpdate call.
+   * Not implemented yet.
+   */
+  Response pauseJobUpdate(1: JobKey jobKey, 2: SessionKey session)
 
-  /** Resumes progress of a previously paused update. Not implemented yet. */
-  Response resumeJobUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
+  /** Resumes progress of a previously paused job update. Not implemented yet. */
+  Response resumeJobUpdate(1: JobKey jobKey, 2: SessionKey session)
 
-  /** Permanently aborts the update. Does not remove the update history. Not implemented yet. */
-  Response abortJobUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
+  /** Permanently aborts the job update. Does not remove the update history. Not implemented yet. */
+  Response abortJobUpdate(1: JobKey jobKey, 2: SessionKey session)
 }
 
 struct InstanceConfigRewrite {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f08c648b/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 649afa2..04abeeb 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -1819,7 +1819,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     control.replay();
 
     Response response =
-        assertOkResponse(thrift.startJobUpdate(request, LOCK.newBuilder(), SESSION));
+        assertOkResponse(thrift.startJobUpdate(request, SESSION));
     assertEquals(UPDATE_ID, response.getResult().getStartJobUpdateResult().getUpdateId());
   }
 
@@ -1829,7 +1829,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     expectAuth(ROLE, false);
 
     control.replay();
-    assertResponse(AUTH_FAILED, thrift.startJobUpdate(request, LOCK.newBuilder(), SESSION));
+    assertResponse(AUTH_FAILED, thrift.startJobUpdate(request, SESSION));
   }
 
   @Test
@@ -1838,7 +1838,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     expectAuth(ROLE, true);
 
     control.replay();
-    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, LOCK.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, SESSION));
   }
 
   @Test
@@ -1849,7 +1849,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(LOCK_ERROR, thrift.startJobUpdate(request, LOCK.newBuilder(), SESSION));
+    assertResponse(LOCK_ERROR, thrift.startJobUpdate(request, SESSION));
   }
 
   @Test
@@ -1862,7 +1862,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, LOCK.newBuilder(), SESSION));
+    assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, SESSION));
   }
 
   private static JobUpdateRequest createJobRequest(TaskConfig config) {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f08c648b/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java b/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
index 38bc9ed..93f79d7 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
@@ -266,27 +266,27 @@ abstract class ForwardingThrift implements AuroraAdmin.Iface {
   }
 
   @Override
-  public Response startJobUpdate(JobUpdateRequest request, Lock lock, SessionKey session)
+  public Response startJobUpdate(JobUpdateRequest request, SessionKey session)
       throws TException {
 
-    return delegate.startJobUpdate(request, lock, session);
+    return delegate.startJobUpdate(request, session);
   }
 
   @Override
-  public Response pauseJobUpdate(String updateId, Lock lock, SessionKey session) throws TException {
-    return delegate.pauseJobUpdate(updateId, lock, session);
+  public Response pauseJobUpdate(JobKey jobKey, SessionKey session) throws TException {
+    return delegate.pauseJobUpdate(jobKey, session);
   }
 
   @Override
-  public Response resumeJobUpdate(String updateId, Lock lock, SessionKey session)
+  public Response resumeJobUpdate(JobKey jobKey, SessionKey session)
       throws TException {
 
-    return delegate.resumeJobUpdate(updateId, lock, session);
+    return delegate.resumeJobUpdate(jobKey, session);
   }
 
   @Override
-  public Response abortJobUpdate(String updateId, Lock lock, SessionKey session) throws TException {
-    return delegate.abortJobUpdate(updateId, lock, session);
+  public Response abortJobUpdate(JobKey jobKey, SessionKey session) throws TException {
+    return delegate.abortJobUpdate(jobKey, session);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f08c648b/src/test/python/apache/aurora/client/api/test_api.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_api.py b/src/test/python/apache/aurora/client/api/test_api.py
index 96db25d..237d4ab 100644
--- a/src/test/python/apache/aurora/client/api/test_api.py
+++ b/src/test/python/apache/aurora/client/api/test_api.py
@@ -135,12 +135,11 @@ class TestJobUpdateApis(unittest.TestCase):
   def test_start_job_update(self):
     """Test successful job update start."""
     api, mock_proxy = self.mock_api()
-    lock = Lock()
     task_config = TaskConfig()
     mock_proxy.startJobUpdate.return_value = self.create_simple_success_response()
 
-    api.start_job_update(self.mock_job_config(), lock)
-    mock_proxy.startJobUpdate.assert_called_once_with(self.create_update_request(task_config), lock)
+    api.start_job_update(self.mock_job_config())
+    mock_proxy.startJobUpdate.assert_called_once_with(self.create_update_request(task_config))
 
   def test_start_job_update_fails_parse_update_config(self):
     """Test start_job_update fails to parse invalid UpdateConfig."""
@@ -149,4 +148,4 @@ class TestJobUpdateApis(unittest.TestCase):
     self.assertRaises(
         AuroraClientAPI.UpdateConfigError,
         api.start_job_update,
-        self.mock_job_config(error=ValueError()), None)
+        self.mock_job_config(error=ValueError()))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f08c648b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_scheduler_client.py b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
index ab74db3..630f662 100644
--- a/src/test/python/apache/aurora/client/api/test_scheduler_client.py
+++ b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
@@ -263,24 +263,24 @@ class TestSchedulerProxyAdminInjection(TestSchedulerProxyInjection):
     self.make_scheduler_proxy().getJobUpdateDetails('update_id')
 
   def test_startJobUpdate(self):
-    self.mock_thrift_client.startJobUpdate(IsA(JobUpdateRequest), IsA(Lock), IsA(SessionKey))
+    self.mock_thrift_client.startJobUpdate(IsA(JobUpdateRequest), IsA(SessionKey))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().startJobUpdate(JobUpdateRequest(), Lock())
+    self.make_scheduler_proxy().startJobUpdate(JobUpdateRequest())
 
   def test_pauseJobUpdate(self):
-    self.mock_thrift_client.pauseJobUpdate('update_id', IsA(Lock), IsA(SessionKey))
+    self.mock_thrift_client.pauseJobUpdate('update_id', IsA(SessionKey))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().pauseJobUpdate('update_id', Lock())
+    self.make_scheduler_proxy().pauseJobUpdate('update_id')
 
   def test_resumeJobUpdate(self):
-    self.mock_thrift_client.resumeJobUpdate('update_id', IsA(Lock), IsA(SessionKey))
+    self.mock_thrift_client.resumeJobUpdate('update_id', IsA(SessionKey))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().resumeJobUpdate('update_id', Lock())
+    self.make_scheduler_proxy().resumeJobUpdate('update_id')
 
   def test_abortJobUpdate(self):
-    self.mock_thrift_client.abortJobUpdate('update_id', IsA(Lock), IsA(SessionKey))
+    self.mock_thrift_client.abortJobUpdate('update_id', IsA(SessionKey))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().abortJobUpdate('update_id', Lock())
+    self.make_scheduler_proxy().abortJobUpdate('update_id')
 
 
 @pytest.mark.parametrize('scheme', ('http', 'https'))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f08c648b/src/test/resources/org/apache/aurora/gen/api.thrift.md5
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/gen/api.thrift.md5 b/src/test/resources/org/apache/aurora/gen/api.thrift.md5
index 0be4d96..81d1734 100644
--- a/src/test/resources/org/apache/aurora/gen/api.thrift.md5
+++ b/src/test/resources/org/apache/aurora/gen/api.thrift.md5
@@ -1 +1 @@
-53397dbf466d235e9d17b52ecd912bbc
+0f59bcf0f25edd7f7eaee987e35ea549