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/19 21:02:24 UTC

git commit: Implementing job update "get" thrift APIs.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master b667ead4b -> f560cad38


Implementing job update "get" thrift APIs.

Bugs closed: AURORA-649

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


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

Branch: refs/heads/master
Commit: f560cad382e2da7d4f62551eab6ce530b8794812
Parents: b667ead
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Tue Aug 19 12:02:07 2014 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Tue Aug 19 12:02:07 2014 -0700

----------------------------------------------------------------------
 .../thrift/SchedulerThriftInterface.java        | 37 ++++++++++--
 .../scheduler/storage/log/LogStorageTest.java   |  8 +--
 .../storage/log/SnapshotStoreImplTest.java      | 10 ++--
 .../storage/testing/StorageTestUtil.java        |  8 +--
 .../thrift/SchedulerThriftInterfaceTest.java    | 62 ++++++++++++++++++++
 .../updater/JobUpdateEventSubscriberTest.java   |  9 +--
 6 files changed, 113 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f560cad3/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 487a1c8..f679f38 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -58,6 +58,8 @@ import org.apache.aurora.gen.ConfigSummary;
 import org.apache.aurora.gen.ConfigSummaryResult;
 import org.apache.aurora.gen.DrainHostsResult;
 import org.apache.aurora.gen.EndMaintenanceResult;
+import org.apache.aurora.gen.GetJobUpdateDetailsResult;
+import org.apache.aurora.gen.GetJobUpdateSummariesResult;
 import org.apache.aurora.gen.GetJobsResult;
 import org.apache.aurora.gen.GetLocksResult;
 import org.apache.aurora.gen.GetPendingReasonResult;
@@ -129,6 +131,8 @@ import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork;
 import org.apache.aurora.scheduler.storage.Storage.NonVolatileStorage;
+import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
+import org.apache.aurora.scheduler.storage.Storage.Work;
 import org.apache.aurora.scheduler.storage.backup.Recovery;
 import org.apache.aurora.scheduler.storage.backup.Recovery.RecoveryException;
 import org.apache.aurora.scheduler.storage.backup.StorageBackup;
@@ -136,7 +140,10 @@ import org.apache.aurora.scheduler.storage.entities.IAssignedTask;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateQuery;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateRequest;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary;
 import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
@@ -1433,13 +1440,35 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
   }
 
   @Override
-  public Response getJobUpdateSummaries(JobUpdateQuery updateQuery) throws TException {
-    throw new UnsupportedOperationException("Not implemented");
+  public Response getJobUpdateSummaries(final JobUpdateQuery mutableQuery) {
+    final IJobUpdateQuery query = IJobUpdateQuery.build(requireNonNull(mutableQuery));
+    return okResponse(Result.getJobUpdateSummariesResult(
+        new GetJobUpdateSummariesResult().setUpdateSummaries(IJobUpdateSummary.toBuildersList(
+            storage.weaklyConsistentRead(new Work.Quiet<List<IJobUpdateSummary>>() {
+              @Override
+              public List<IJobUpdateSummary> apply(StoreProvider storeProvider) {
+                return storeProvider.getJobUpdateStore().fetchJobUpdateSummaries(query);
+              }
+            })))));
   }
 
   @Override
-  public Response getJobUpdateDetails(String updateId) {
-    throw new UnsupportedOperationException("Not implemented");
+  public Response getJobUpdateDetails(final String updateId) {
+    requireNonNull(updateId);
+    Optional<IJobUpdateDetails> details =
+        storage.weaklyConsistentRead(new Work.Quiet<Optional<IJobUpdateDetails>>() {
+          @Override
+          public Optional<IJobUpdateDetails> apply(StoreProvider storeProvider) {
+            return storeProvider.getJobUpdateStore().fetchJobUpdateDetails(updateId);
+          }
+        });
+
+    if (details.isPresent()) {
+      return okResponse(Result.getJobUpdateDetailsResult(
+          new GetJobUpdateDetailsResult().setDetails(details.get().newBuilder())));
+    } else {
+      return addMessage(emptyResponse(), INVALID_REQUEST, "Invalid update ID:" + updateId);
+    }
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f560cad3/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java b/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
index 339774c..ca990e7 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
@@ -149,7 +149,7 @@ public class LogStorageTest extends EasyMockTest {
             storageUtil.lockStore,
             storageUtil.quotaStore,
             storageUtil.attributeStore,
-            storageUtil.updateStore);
+            storageUtil.jobUpdateStore);
 
     stream = createMock(Stream.class);
     streamMatcher = LogOpMatcher.matcherFor(stream);
@@ -769,7 +769,7 @@ public class LogStorageTest extends EasyMockTest {
       @Override
       protected void setupExpectations() throws Exception {
         storageUtil.expectWriteOperation();
-        storageUtil.updateStore.saveJobUpdate(update, lockToken);
+        storageUtil.jobUpdateStore.saveJobUpdate(update, lockToken);
         streamMatcher.expectTransaction(
             Op.saveJobUpdate(new SaveJobUpdate(update.newBuilder(), lockToken)))
             .andReturn(position);
@@ -792,7 +792,7 @@ public class LogStorageTest extends EasyMockTest {
       @Override
       protected void setupExpectations() throws Exception {
         storageUtil.expectWriteOperation();
-        storageUtil.updateStore.saveJobUpdateEvent(event, UPDATE_ID);
+        storageUtil.jobUpdateStore.saveJobUpdateEvent(event, UPDATE_ID);
         streamMatcher.expectTransaction(Op.saveJobUpdateEvent(new SaveJobUpdateEvent(
             event.newBuilder(),
             UPDATE_ID))).andReturn(position);
@@ -816,7 +816,7 @@ public class LogStorageTest extends EasyMockTest {
       @Override
       protected void setupExpectations() throws Exception {
         storageUtil.expectWriteOperation();
-        storageUtil.updateStore.saveJobInstanceUpdateEvent(event, UPDATE_ID);
+        storageUtil.jobUpdateStore.saveJobInstanceUpdateEvent(event, UPDATE_ID);
         streamMatcher.expectTransaction(Op.saveJobInstanceUpdateEvent(
             new SaveJobInstanceUpdateEvent(event.newBuilder(), UPDATE_ID))).andReturn(position);
       }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f560cad3/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
index c752e66..6de7e19 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
@@ -119,7 +119,7 @@ public class SnapshotStoreImplTest extends EasyMockTest {
     expect(storageUtil.schedulerStore.fetchFrameworkId()).andReturn(Optional.of(frameworkId));
     expect(storageUtil.lockStore.fetchLocks()).andReturn(ImmutableSet.of(lock));
     String lockToken = "token";
-    expect(storageUtil.updateStore.fetchAllJobUpdateDetails())
+    expect(storageUtil.jobUpdateStore.fetchAllJobUpdateDetails())
         .andReturn(ImmutableSet.of(
             new StoredJobUpdateDetails(updateDetails.newBuilder(), lockToken)));
 
@@ -132,11 +132,11 @@ public class SnapshotStoreImplTest extends EasyMockTest {
         IJobConfiguration.build(job.getJobConfiguration()));
     storageUtil.schedulerStore.saveFrameworkId(frameworkId);
     storageUtil.lockStore.saveLock(lock);
-    storageUtil.updateStore.saveJobUpdate(updateDetails.getUpdate(), lockToken);
-    storageUtil.updateStore.saveJobUpdateEvent(
+    storageUtil.jobUpdateStore.saveJobUpdate(updateDetails.getUpdate(), lockToken);
+    storageUtil.jobUpdateStore.saveJobUpdateEvent(
         Iterables.getOnlyElement(updateDetails.getUpdateEvents()),
         updateId);
-    storageUtil.updateStore.saveJobInstanceUpdateEvent(
+    storageUtil.jobUpdateStore.saveJobInstanceUpdateEvent(
         Iterables.getOnlyElement(updateDetails.getInstanceEvents()),
         updateId);
 
@@ -164,6 +164,6 @@ public class SnapshotStoreImplTest extends EasyMockTest {
     storageUtil.attributeStore.deleteHostAttributes();
     storageUtil.jobStore.deleteJobs();
     storageUtil.lockStore.deleteLocks();
-    storageUtil.updateStore.deleteAllUpdatesAndEvents();
+    storageUtil.jobUpdateStore.deleteAllUpdatesAndEvents();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f560cad3/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java b/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java
index 4bad839..5aebbfb 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java
@@ -54,7 +54,7 @@ public class StorageTestUtil {
   public final JobStore.Mutable jobStore;
   public final LockStore.Mutable lockStore;
   public final SchedulerStore.Mutable schedulerStore;
-  public final JobUpdateStore.Mutable updateStore;
+  public final JobUpdateStore.Mutable jobUpdateStore;
   public final NonVolatileStorage storage;
 
   /**
@@ -71,7 +71,7 @@ public class StorageTestUtil {
     this.jobStore = easyMock.createMock(JobStore.Mutable.class);
     this.lockStore = easyMock.createMock(LockStore.Mutable.class);
     this.schedulerStore = easyMock.createMock(SchedulerStore.Mutable.class);
-    this.updateStore = easyMock.createMock(JobUpdateStore.Mutable.class);
+    this.jobUpdateStore = easyMock.createMock(JobUpdateStore.Mutable.class);
     this.storage = easyMock.createMock(NonVolatileStorage.class);
   }
 
@@ -117,7 +117,7 @@ public class StorageTestUtil {
     expect(storeProvider.getJobStore()).andReturn(jobStore).anyTimes();
     expect(storeProvider.getLockStore()).andReturn(lockStore).anyTimes();
     expect(storeProvider.getSchedulerStore()).andReturn(schedulerStore).anyTimes();
-    expect(storeProvider.getJobUpdateStore()).andReturn(updateStore).anyTimes();
+    expect(storeProvider.getJobUpdateStore()).andReturn(jobUpdateStore).anyTimes();
     expect(mutableStoreProvider.getTaskStore()).andReturn(taskStore).anyTimes();
     expect(mutableStoreProvider.getUnsafeTaskStore()).andReturn(taskStore).anyTimes();
     expect(mutableStoreProvider.getQuotaStore()).andReturn(quotaStore).anyTimes();
@@ -125,7 +125,7 @@ public class StorageTestUtil {
     expect(mutableStoreProvider.getJobStore()).andReturn(jobStore).anyTimes();
     expect(mutableStoreProvider.getLockStore()).andReturn(lockStore).anyTimes();
     expect(mutableStoreProvider.getSchedulerStore()).andReturn(schedulerStore).anyTimes();
-    expect(mutableStoreProvider.getJobUpdateStore()).andReturn(updateStore).anyTimes();
+    expect(mutableStoreProvider.getJobUpdateStore()).andReturn(jobUpdateStore).anyTimes();
     expectConsistentRead().anyTimes();
     expectWeaklyConsistentRead().anyTimes();
     expectWriteOperation().anyTimes();

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f560cad3/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 48c8078..997ade0 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -67,6 +67,8 @@ import org.apache.aurora.gen.JobSummary;
 import org.apache.aurora.gen.JobSummaryResult;
 import org.apache.aurora.gen.JobUpdate;
 import org.apache.aurora.gen.JobUpdateConfiguration;
+import org.apache.aurora.gen.JobUpdateDetails;
+import org.apache.aurora.gen.JobUpdateQuery;
 import org.apache.aurora.gen.JobUpdateRequest;
 import org.apache.aurora.gen.JobUpdateSettings;
 import org.apache.aurora.gen.JobUpdateSummary;
@@ -119,6 +121,9 @@ import org.apache.aurora.scheduler.storage.backup.StorageBackup;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateQuery;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary;
 import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
@@ -1825,6 +1830,63 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
   }
 
   @Test
+  public void testGetJobUpdateSummaries() throws Exception {
+    JobUpdateQuery query = new JobUpdateQuery().setRole(ROLE);
+    List<JobUpdateSummary> summaries = createJobUpdateSummaries(5);
+    expect(storageUtil.jobUpdateStore.fetchJobUpdateSummaries(IJobUpdateQuery.build(query)))
+        .andReturn(IJobUpdateSummary.listFromBuilders(summaries));
+
+    control.replay();
+
+    Response response = assertOkResponse(thrift.getJobUpdateSummaries(query));
+    assertEquals(
+        summaries,
+        response.getResult().getGetJobUpdateSummariesResult().getUpdateSummaries());
+  }
+
+  @Test
+  public void testGetJobUpdateDetails() throws Exception {
+    String id = "id";
+    JobUpdateDetails details = createJobUpdateDetails();
+    expect(storageUtil.jobUpdateStore.fetchJobUpdateDetails(id))
+        .andReturn(Optional.of(IJobUpdateDetails.build(details)));
+
+    control.replay();
+
+    Response response = assertOkResponse(thrift.getJobUpdateDetails(id));
+    assertEquals(
+        details,
+        response.getResult().getGetJobUpdateDetailsResult().getDetails());
+  }
+
+  @Test
+  public void testGetJobUpdateDetailsInvalidId() throws Exception {
+    String id = "id";
+    expect(storageUtil.jobUpdateStore.fetchJobUpdateDetails(id))
+        .andReturn(Optional.<IJobUpdateDetails>absent());
+
+    control.replay();
+
+    assertResponse(INVALID_REQUEST, thrift.getJobUpdateDetails(id));
+  }
+
+  private static List<JobUpdateSummary> createJobUpdateSummaries(int count) {
+    ImmutableList.Builder<JobUpdateSummary> builder = ImmutableList.builder();
+    for (int i = 0; i < count; i++) {
+      builder.add(new JobUpdateSummary()
+          .setUpdateId("id" + i)
+          .setJobKey(JOB_KEY.newBuilder())
+          .setUser(USER));
+    }
+    return builder.build();
+  }
+
+  private static JobUpdateDetails createJobUpdateDetails() {
+    return new JobUpdateDetails()
+        .setUpdate(new JobUpdate().setSummary(createJobUpdateSummaries(1).get(0)));
+  }
+
+  @Test
   public void testStartUpdate() throws Exception {
     expectAuth(ROLE, true);
     expect(lockManager.acquireLock(LOCK_KEY, USER)).andReturn(LOCK);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/f560cad3/src/test/java/org/apache/aurora/scheduler/updater/JobUpdateEventSubscriberTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/updater/JobUpdateEventSubscriberTest.java b/src/test/java/org/apache/aurora/scheduler/updater/JobUpdateEventSubscriberTest.java
index 2a7d965..4127442 100644
--- a/src/test/java/org/apache/aurora/scheduler/updater/JobUpdateEventSubscriberTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/updater/JobUpdateEventSubscriberTest.java
@@ -91,8 +91,9 @@ public class JobUpdateEventSubscriberTest extends EasyMockTest {
 
   @Test
   public void testSchedulerStartup() throws Exception {
-    expect(storageUtil.updateStore.fetchJobUpdateSummaries(JobUpdateEventSubscriber.ACTIVE_QUERY))
-        .andReturn(ImmutableList.of(SUMMARY));
+    expect(storageUtil.jobUpdateStore.fetchJobUpdateSummaries(
+        JobUpdateEventSubscriber.ACTIVE_QUERY)).andReturn(ImmutableList.of(SUMMARY));
+
     updater.systemResume(JOB_A);
 
     control.replay();
@@ -102,8 +103,8 @@ public class JobUpdateEventSubscriberTest extends EasyMockTest {
 
   @Test
   public void testSchedulerStartupNoUpdates() throws Exception {
-    expect(storageUtil.updateStore.fetchJobUpdateSummaries(JobUpdateEventSubscriber.ACTIVE_QUERY))
-        .andReturn(ImmutableList.<IJobUpdateSummary>of());
+    expect(storageUtil.jobUpdateStore.fetchJobUpdateSummaries(
+        JobUpdateEventSubscriber.ACTIVE_QUERY)).andReturn(ImmutableList.<IJobUpdateSummary>of());
 
     control.replay();