You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by zm...@apache.org on 2016/09/13 19:45:58 UTC

aurora git commit: Extend getJobUpdateDetails to accept JobUpdateQuery

Repository: aurora
Updated Branches:
  refs/heads/master f0ff1c4fb -> 795a2728c


Extend getJobUpdateDetails to accept JobUpdateQuery

This extends getJobUpdateDetails to return a list of details instead of being
scoped to a single update.

Bugs closed: AURORA-1764

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


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

Branch: refs/heads/master
Commit: 795a2728c623c35bd509d582c24684a6921c74ad
Parents: f0ff1c4
Author: Zameer Manji <zm...@apache.org>
Authored: Tue Sep 13 12:45:17 2016 -0700
Committer: Zameer Manji <zm...@apache.org>
Committed: Tue Sep 13 12:45:17 2016 -0700

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |  1 +
 .../thrift/org/apache/aurora/gen/api.thrift     |  5 ++-
 .../scheduler/thrift/ReadOnlySchedulerImpl.java | 27 +++++++++++++---
 .../thrift/SchedulerThriftInterface.java        |  4 +--
 .../python/apache/aurora/client/api/__init__.py |  4 ++-
 .../python/apache/aurora/client/cli/update.py   | 11 ++++++-
 .../resources/scheduler/assets/js/services.js   |  7 +++--
 .../thrift/ReadOnlySchedulerImplTest.java       | 33 +++++++++++++++++---
 .../python/apache/aurora/client/api/test_api.py |  3 +-
 .../apache/aurora/client/cli/test_supdate.py    |  3 +-
 10 files changed, 80 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index 4476d52..b947c23 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -48,6 +48,7 @@
   It will be removed from Mesos in a future release.
 - The scheduler flag `-zk_use_curator` has been deprecated. If you have never set the flag and are
   upgrading you should take care as described in the [note](#zk_use_curator_upgrade) above.
+- The `key` argument of `getJobUpdateDetails` has been deprecated. Use the `query` argument instead.
 
 0.15.0
 ======

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/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 c5765b7..3f7079d 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -896,7 +896,9 @@ struct GetJobUpdateSummariesResult {
 
 /** Result of the getJobUpdateDetails call. */
 struct GetJobUpdateDetailsResult {
+  // TODO(zmanji): Remove this once we complete AURORA-1765
   1: JobUpdateDetails details
+  2: list<JobUpdateDetails> detailsList
 }
 
 /** Result of the pulseJobUpdate call. */
@@ -1022,7 +1024,8 @@ service ReadOnlyScheduler {
   Response getJobUpdateSummaries(1: JobUpdateQuery jobUpdateQuery)
 
   /** Gets job update details. */
-  Response getJobUpdateDetails(1: JobUpdateKey key)
+  // TODO(zmanji): `key` is deprecated, remove this with AURORA-1765
+  Response getJobUpdateDetails(1: JobUpdateKey key, 2: JobUpdateQuery query)
 
   /** Gets the diff between client (desired) and server (current) job states. */
   Response getJobUpdateDiff(1: JobUpdateRequest request)

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java b/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java
index 4202f0e..bba1161 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java
@@ -108,6 +108,7 @@ import static org.apache.aurora.gen.ResponseCode.INVALID_REQUEST;
 import static org.apache.aurora.scheduler.base.Numbers.convertRanges;
 import static org.apache.aurora.scheduler.base.Numbers.toRanges;
 import static org.apache.aurora.scheduler.resources.ResourceManager.aggregateFromBag;
+import static org.apache.aurora.scheduler.thrift.Responses.addMessage;
 import static org.apache.aurora.scheduler.thrift.Responses.error;
 import static org.apache.aurora.scheduler.thrift.Responses.invalidRequest;
 import static org.apache.aurora.scheduler.thrift.Responses.ok;
@@ -315,14 +316,30 @@ class ReadOnlySchedulerImpl implements ReadOnlyScheduler.Iface {
   }
 
   @Override
-  public Response getJobUpdateDetails(JobUpdateKey mutableKey) {
+  public Response getJobUpdateDetails(JobUpdateKey mutableKey, JobUpdateQuery mutableQuery) {
+    if (mutableKey == null && mutableQuery == null)  {
+      return error("Either key or query must be set.");
+    }
+
+    if (mutableQuery != null) {
+      IJobUpdateQuery query = IJobUpdateQuery.build(mutableQuery);
+
+      List<IJobUpdateDetails> details = storage.read(storeProvider ->
+          storeProvider.getJobUpdateStore().fetchJobUpdateDetails(query));
+
+      return ok(Result.getJobUpdateDetailsResult(new GetJobUpdateDetailsResult()
+          .setDetailsList(IJobUpdateDetails.toBuildersList(details))));
+    }
+
+    // TODO(zmanji): Remove this code once `mutableKey` is removed in AURORA-1765
     IJobUpdateKey key = IJobUpdateKey.build(mutableKey);
-    Optional<IJobUpdateDetails> details =
-        storage.read(storeProvider -> storeProvider.getJobUpdateStore().fetchJobUpdateDetails(key));
+    Optional<IJobUpdateDetails> details = storage.read(storeProvider ->
+        storeProvider.getJobUpdateStore().fetchJobUpdateDetails(key));
 
     if (details.isPresent()) {
-      return ok(Result.getJobUpdateDetailsResult(
-          new GetJobUpdateDetailsResult().setDetails(details.get().newBuilder())));
+      return addMessage(ok(Result.getJobUpdateDetailsResult(
+          new GetJobUpdateDetailsResult().setDetails(details.get().newBuilder()))),
+          "The key argument is deprecated, use the query argument instead");
     } else {
       return invalidRequest("Invalid update: " + key);
     }

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/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 929d021..cd18d0d 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -1003,8 +1003,8 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response getJobUpdateDetails(JobUpdateKey key) throws TException {
-    return readOnlyScheduler.getJobUpdateDetails(key);
+  public Response getJobUpdateDetails(JobUpdateKey key, JobUpdateQuery query) throws TException {
+    return readOnlyScheduler.getJobUpdateDetails(key, query);
   }
 
   private static IJobUpdateKey validateJobUpdateKey(JobUpdateKey mutableKey) {

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/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 9149c30..05b9c3b 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -261,7 +261,9 @@ class AuroraClientAPI(object):
     if not isinstance(key, JobUpdateKey):
       raise self.TypeError('Invalid key %r: expected %s but got %s'
                            % (key, JobUpdateKey.__name__, key.__class__.__name__))
-    return self._scheduler_proxy.getJobUpdateDetails(key)
+
+    query = JobUpdateQuery(key=key)
+    return self._scheduler_proxy.getJobUpdateDetails(key, query)
 
   def restart(self, job_key, instances, restart_settings):
     """Perform a rolling restart of the job.

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/src/main/python/apache/aurora/client/cli/update.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/update.py b/src/main/python/apache/aurora/client/cli/update.py
index d23243d..a634c44 100644
--- a/src/main/python/apache/aurora/client/cli/update.py
+++ b/src/main/python/apache/aurora/client/cli/update.py
@@ -526,7 +526,16 @@ class UpdateInfo(Verb):
     api = context.get_api(context.options.jobspec.cluster)
     response = api.get_job_update_details(key)
     context.log_response_and_raise(response)
-    details = response.result.getJobUpdateDetailsResult.details
+    detailsList = response.result.getJobUpdateDetailsResult.detailsList
+    if detailsList is not None:
+      if len(detailsList) == 0:
+        context.print_err("There is no update for key: %s" % key)
+        return EXIT_INVALID_PARAMETER
+
+      details = detailsList[0]
+    else:
+      details = response.result.getJobUpdateDetailsResult.details
+
     if context.options.write_json:
       result = {
         "updateId": ("%s" % details.update.summary.key.id),

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/src/main/resources/scheduler/assets/js/services.js
----------------------------------------------------------------------
diff --git a/src/main/resources/scheduler/assets/js/services.js b/src/main/resources/scheduler/assets/js/services.js
index 315fc17..0818fc2 100644
--- a/src/main/resources/scheduler/assets/js/services.js
+++ b/src/main/resources/scheduler/assets/js/services.js
@@ -167,10 +167,13 @@
 
           getJobUpdateDetails: function (updateKey) {
             return async(function (deferred) {
-              auroraClient.getSchedulerClient().getJobUpdateDetails(updateKey, function (response) {
+              var schedulerClient = auroraClient.getSchedulerClient();
+              var query = new JobUpdateQuery();
+              query.key = updateKey;
+              schedulerClient.getJobUpdateDetails(null, query, function (response) {
                 var result = auroraClient.processResponse(response);
                 result.details = response.result !== null ?
-                  response.result.getJobUpdateDetailsResult.details : {};
+                  response.result.getJobUpdateDetailsResult.detailsList[0] : {};
                 deferred.resolve(result);
               });
             });

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
index a7d1f74..7935432 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
@@ -17,6 +17,7 @@ import java.util.Date;
 import java.util.List;
 import java.util.Set;
 
+import java.util.stream.Collectors;
 import javax.annotation.Nullable;
 
 import com.google.common.base.Function;
@@ -601,10 +602,27 @@ public class ReadOnlySchedulerImplTest extends EasyMockTest {
 
     control.replay();
 
-    Response response = assertOkResponse(thrift.getJobUpdateDetails(UPDATE_KEY.newBuilder()));
+    Response response = assertOkResponse(thrift.getJobUpdateDetails(UPDATE_KEY.newBuilder(), null));
     assertEquals(
         IJobUpdateDetails.build(details),
         IJobUpdateDetails.build(response.getResult().getGetJobUpdateDetailsResult().getDetails()));
+    // Specifying the UPDATE_KEY alone is deprecated, so there should be a message.
+    assertEquals(1, response.getDetailsSize());
+  }
+
+  @Test
+  public void testGetJobUpdateDetailsQuery() throws Exception {
+    JobUpdateQuery query = new JobUpdateQuery().setRole(ROLE);
+    List<IJobUpdateDetails> details = IJobUpdateDetails.listFromBuilders(createJobUpdateDetails(5));
+    expect(storageUtil.jobUpdateStore.fetchJobUpdateDetails(IJobUpdateQuery.build(query)))
+        .andReturn(details);
+
+    control.replay();
+
+    Response response = assertOkResponse(thrift.getJobUpdateDetails(null, query));
+    assertEquals(
+        IJobUpdateDetails.toBuildersList(details),
+        response.getResult().getGetJobUpdateDetailsResult().getDetailsList());
   }
 
   private static List<JobUpdateSummary> createJobUpdateSummaries(int count) {
@@ -617,9 +635,16 @@ public class ReadOnlySchedulerImplTest extends EasyMockTest {
     return builder.build();
   }
 
+  private static List<JobUpdateDetails> createJobUpdateDetails(int count) {
+    List<JobUpdateSummary> summaries = createJobUpdateSummaries(count);
+    return summaries.stream()
+        .map(jobUpdateSummary ->
+            new JobUpdateDetails().setUpdate(new JobUpdate().setSummary(jobUpdateSummary)))
+        .collect(Collectors.toList());
+  }
+
   private static JobUpdateDetails createJobUpdateDetails() {
-    return new JobUpdateDetails()
-        .setUpdate(new JobUpdate().setSummary(createJobUpdateSummaries(1).get(0)));
+    return createJobUpdateDetails(1).get(0);
   }
 
   @Test
@@ -706,7 +731,7 @@ public class ReadOnlySchedulerImplTest extends EasyMockTest {
 
     control.replay();
 
-    assertResponse(INVALID_REQUEST, thrift.getJobUpdateDetails(UPDATE_KEY.newBuilder()));
+    assertResponse(INVALID_REQUEST, thrift.getJobUpdateDetails(UPDATE_KEY.newBuilder(), null));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/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 7a0797d..f1579b5 100644
--- a/src/test/python/apache/aurora/client/api/test_api.py
+++ b/src/test/python/apache/aurora/client/api/test_api.py
@@ -190,7 +190,8 @@ class TestJobUpdateApis(unittest.TestCase):
     api, mock_proxy = self.mock_api()
     key = JobUpdateKey(job=JobKey(role="role", environment="env", name="name"), id="id")
     api.get_job_update_details(key)
-    mock_proxy.getJobUpdateDetails.assert_called_once_with(key)
+    query = JobUpdateQuery(key=key)
+    mock_proxy.getJobUpdateDetails.assert_called_once_with(key, query)
 
   def test_set_quota(self):
     """Test setting quota."""

http://git-wip-us.apache.org/repos/asf/aurora/blob/795a2728/src/test/python/apache/aurora/client/cli/test_supdate.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_supdate.py b/src/test/python/apache/aurora/client/cli/test_supdate.py
index 92fb039..2dfee1f 100644
--- a/src/test/python/apache/aurora/client/cli/test_supdate.py
+++ b/src/test/python/apache/aurora/client/cli/test_supdate.py
@@ -553,7 +553,8 @@ class TestUpdateInfo(AuroraClientCommandTest):
                 instanceId=2,
                 timestampMs=9000,
                 action=JobUpdateAction.INSTANCE_UPDATED)])
-    query_response.result.getJobUpdateDetailsResult = GetJobUpdateDetailsResult(details=details)
+    query_response.result.getJobUpdateDetailsResult = GetJobUpdateDetailsResult(
+            detailsList=[details])
     return query_response
 
   def test_active_update_info(self):