Return-Path: X-Original-To: apmail-aurora-commits-archive@minotaur.apache.org Delivered-To: apmail-aurora-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 0F87710FD0 for ; Tue, 11 Mar 2014 06:27:37 +0000 (UTC) Received: (qmail 40063 invoked by uid 500); 11 Mar 2014 06:27:36 -0000 Delivered-To: apmail-aurora-commits-archive@aurora.apache.org Received: (qmail 40028 invoked by uid 500); 11 Mar 2014 06:27:32 -0000 Mailing-List: contact commits-help@aurora.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@aurora.incubator.apache.org Delivered-To: mailing list commits@aurora.incubator.apache.org Received: (qmail 40021 invoked by uid 99); 11 Mar 2014 06:27:31 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 11 Mar 2014 06:27:31 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Tue, 11 Mar 2014 06:27:24 +0000 Received: (qmail 39948 invoked by uid 99); 11 Mar 2014 06:27:01 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 11 Mar 2014 06:27:01 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 8F5BB93E51D; Tue, 11 Mar 2014 06:27:00 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: mansu@apache.org To: commits@aurora.incubator.apache.org Message-Id: <633afa1d24d541a89af3aeda3a51f236@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: git commit: Added getJobSummary API so it can be used by the role and role/environment page in the UI. Date: Tue, 11 Mar 2014 06:27:00 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Repository: incubator-aurora Updated Branches: refs/heads/master 9642eee84 -> d96d4bbf3 Added getJobSummary API so it can be used by the role and role/environment page in the UI. Refactored code from SchedulerzRole and SchedulerzRoleTest into relevant classes so it can be used by the UI and the thrift API. Added tests for new code. Moved populateJobConfig call into ReadOnlyScheduler. Testing Done: gradle clean build gradle run to test local UI. Bugs closed: AURORA-64 Reviewed at https://reviews.apache.org/r/17303/ Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/d96d4bbf Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/d96d4bbf Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/d96d4bbf Branch: refs/heads/master Commit: d96d4bbf32f5a38687215de4385d5791a0d3d319 Parents: 9642eee Author: Suman Karumuri Authored: Mon Mar 10 23:23:44 2014 -0700 Committer: Suman Karumuri Committed: Mon Mar 10 23:23:44 2014 -0700 ---------------------------------------------------------------------- .../org/apache/aurora/scheduler/base/Jobs.java | 73 ++++++++++ .../org/apache/aurora/scheduler/base/Tasks.java | 26 ++++ .../aurora/scheduler/http/SchedulerzRole.java | 72 +--------- .../thrift/SchedulerThriftInterface.java | 136 +++++++++++-------- .../thrift/org/apache/aurora/gen/api.thrift | 28 +++- .../apache/aurora/scheduler/base/JobsTest.java | 55 ++++++++ .../aurora/scheduler/base/TaskTestUtil.java | 68 ++++++++++ .../apache/aurora/scheduler/base/TasksTest.java | 83 +++++++++++ .../scheduler/http/SchedulerzRoleTest.java | 95 ------------- .../thrift/SchedulerThriftInterfaceTest.java | 116 +++++++++++++++- .../scheduler/thrift/aop/ForwardingThrift.java | 5 + .../org/apache/aurora/gen/api.thrift.md5 | 2 +- 12 files changed, 529 insertions(+), 230 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/src/main/java/org/apache/aurora/scheduler/base/Jobs.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/base/Jobs.java b/src/main/java/org/apache/aurora/scheduler/base/Jobs.java new file mode 100644 index 0000000..3f02703 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/base/Jobs.java @@ -0,0 +1,73 @@ +/** + * Copyright 2014 Apache Software Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.aurora.scheduler.base; + +import org.apache.aurora.gen.JobStats; +import org.apache.aurora.scheduler.storage.entities.IScheduledTask; + +/** + * Convenience methods related to jobs. + */ +public final class Jobs { + + private Jobs() { + // Utility class. + } + + /** + * For a given collection of tasks compute statistics based on the state of the task. + * + * @param tasks a collection of tasks for which statistics are sought + * @return an JobStats object containing the statistics about the tasks. + */ + public static JobStats getJobStats(Iterable tasks) { + JobStats stats = new JobStats(); + for (IScheduledTask task : tasks) { + switch (task.getStatus()) { + case INIT: + case PENDING: + case THROTTLED: + stats.pendingTaskCount++; + break; + + case ASSIGNED: + case STARTING: + case RESTARTING: + case RUNNING: + case KILLING: + case DRAINING: + case PREEMPTING: + stats.activeTaskCount++; + break; + + case KILLED: + case FINISHED: + stats.finishedTaskCount++; + break; + + case LOST: + case FAILED: + case UNKNOWN: + stats.failedTaskCount++; + break; + + default: + throw new IllegalArgumentException("Unsupported status: " + task.getStatus()); + } + } + return stats; + } +} http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/src/main/java/org/apache/aurora/scheduler/base/Tasks.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/base/Tasks.java b/src/main/java/org/apache/aurora/scheduler/base/Tasks.java index d9cb886..003d475 100644 --- a/src/main/java/org/apache/aurora/scheduler/base/Tasks.java +++ b/src/main/java/org/apache/aurora/scheduler/base/Tasks.java @@ -16,11 +16,14 @@ package org.apache.aurora.scheduler.base; import java.util.EnumSet; +import java.util.List; import java.util.Map; import java.util.Set; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -44,6 +47,12 @@ import org.apache.aurora.scheduler.storage.entities.ITaskEvent; */ public final class Tasks { + @VisibleForTesting + static final List ORDERED_TASK_STATUSES = ImmutableList.builder() + .addAll(apiConstants.TERMINAL_STATES) + .addAll(apiConstants.ACTIVE_STATES) + .build(); + public static final Function SCHEDULED_TO_ASSIGNED = new Function() { @Override @@ -146,6 +155,7 @@ public final class Tasks { /** * Order by production flag (true, then false), subsorting by task ID. + * TODO(Suman Karumuri): Move this call into SchedulerThriftInterface once SchedulerzRole is gone. */ public static final Ordering SCHEDULING_ORDER = Ordering.explicit(true, false) @@ -158,6 +168,7 @@ public final class Tasks { /** * A utility method that returns a multi-map of tasks keyed by IJobKey. + * * @param tasks A list of tasks to be keyed by map * @return A multi-map of tasks keyed by job key. */ @@ -202,6 +213,21 @@ public final class Tasks { return task.getAssignedTask().getTask().getJobName(); } + /** + * Get the latest active task or the latest inactive task if no active task exists. + * + * @param tasks a collection of tasks + * @return the task that transitioned most recently. + */ + public static IScheduledTask getLatestActiveTask(Iterable tasks) { + Preconditions.checkArgument(Iterables.size(tasks) != 0); + + return Ordering.explicit(ORDERED_TASK_STATUSES) + .onResultOf(GET_STATUS) + .compound(LATEST_ACTIVITY) + .max(tasks); + } + public static ITaskEvent getLatestEvent(IScheduledTask task) { return Iterables.getLast(task.getTaskEvents()); } http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/src/main/java/org/apache/aurora/scheduler/http/SchedulerzRole.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/http/SchedulerzRole.java b/src/main/java/org/apache/aurora/scheduler/http/SchedulerzRole.java index 60b2259..1151b03 100644 --- a/src/main/java/org/apache/aurora/scheduler/http/SchedulerzRole.java +++ b/src/main/java/org/apache/aurora/scheduler/http/SchedulerzRole.java @@ -16,7 +16,6 @@ package org.apache.aurora.scheduler.http; import java.util.Collection; -import java.util.Date; import java.util.List; import java.util.Map; import java.util.Set; @@ -31,28 +30,21 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Predicate; import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; -import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.twitter.common.base.Closure; -import com.twitter.common.quantity.Amount; -import com.twitter.common.quantity.Time; import org.antlr.stringtemplate.StringTemplate; import org.apache.aurora.gen.CronCollisionPolicy; -import org.apache.aurora.gen.ScheduleStatus; -import org.apache.aurora.gen.apiConstants; import org.apache.aurora.scheduler.base.JobKeys; import org.apache.aurora.scheduler.base.Query; import org.apache.aurora.scheduler.base.Tasks; @@ -70,8 +62,7 @@ import org.apache.aurora.scheduler.storage.entities.ITaskConfig; import static com.google.common.base.Preconditions.checkNotNull; import static com.twitter.common.base.MorePreconditions.checkNotBlank; -import static org.apache.aurora.scheduler.base.Tasks.GET_STATUS; -import static org.apache.aurora.scheduler.base.Tasks.LATEST_ACTIVITY; +import static org.apache.aurora.scheduler.base.Tasks.getLatestActiveTask; /** * HTTP interface to provide information about jobs for a specific role. @@ -79,21 +70,6 @@ import static org.apache.aurora.scheduler.base.Tasks.LATEST_ACTIVITY; @Path("/scheduler/{role}") public class SchedulerzRole extends JerseyTemplateServlet { - private static final List STATUSES = ImmutableList.builder() - .addAll(apiConstants.TERMINAL_STATES) - .addAll(apiConstants.ACTIVE_STATES) - .build(); - - // The freshest task is the latest active task - // or the latest inactive task if no active task exists. - private static final Ordering FRESH_TASK_ORDER = - Ordering.explicit(STATUSES).onResultOf(GET_STATUS).compound(LATEST_ACTIVITY); - - @VisibleForTesting - static IScheduledTask getFreshestTask(Iterable tasks) { - return FRESH_TASK_ORDER.max(tasks); - } - private final Storage storage; private final CronJobManager cronJobManager; private final CronPredictor cronPredictor; @@ -247,11 +223,12 @@ public class SchedulerzRole extends JerseyTemplateServlet { job.name = jobKey.getName(); // Pick the freshest task's config and associate it with the job. - ITaskConfig freshestConfig = getFreshestTask(tasks).getAssignedTask().getTask(); - job.production = freshestConfig.isProduction(); + ITaskConfig mostRecentTaskConfig = + getLatestActiveTask(tasks).getAssignedTask().getTask(); + job.production = mostRecentTaskConfig.isProduction(); // TODO(Suman Karumuri): Add a source/job type to TaskConfig and replace logic below - if (freshestConfig.isIsService()) { + if (mostRecentTaskConfig.isIsService()) { job.type = JobType.SERVICE; } else if (cronJobs.containsKey(jobKey)) { job.type = JobType.CRON; @@ -259,45 +236,6 @@ public class SchedulerzRole extends JerseyTemplateServlet { job.type = JobType.ADHOC; } - for (IScheduledTask task : tasks) { - switch (task.getStatus()) { - case INIT: - case THROTTLED: - case PENDING: - job.pendingTaskCount++; - break; - - case ASSIGNED: - case STARTING: - case RESTARTING: - case DRAINING: - case RUNNING: - case KILLING: - case PREEMPTING: - job.activeTaskCount++; - break; - - case KILLED: - case FINISHED: - job.finishedTaskCount++; - break; - - case LOST: - case FAILED: - case UNKNOWN: - job.failedTaskCount++; - Date now = new Date(); - long elapsedMillis = now.getTime() - Tasks.getLatestEvent(task).getTimestamp(); - if (Amount.of(elapsedMillis, Time.MILLISECONDS).as(Time.HOURS) < 6) { - job.recentlyFailedTaskCount++; - } - break; - - default: - throw new IllegalArgumentException("Unsupported status: " + task.getStatus()); - } - } - return job; } }; http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/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 4911c77..ee81b49 100644 --- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java +++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java @@ -69,6 +69,8 @@ import org.apache.aurora.gen.InstanceKey; import org.apache.aurora.gen.JobConfigRewrite; import org.apache.aurora.gen.JobConfiguration; import org.apache.aurora.gen.JobKey; +import org.apache.aurora.gen.JobSummary; +import org.apache.aurora.gen.JobSummaryResult; import org.apache.aurora.gen.ListBackupsResult; import org.apache.aurora.gen.Lock; import org.apache.aurora.gen.LockKey; @@ -90,6 +92,7 @@ import org.apache.aurora.gen.StartMaintenanceResult; import org.apache.aurora.gen.TaskConfig; import org.apache.aurora.gen.TaskQuery; import org.apache.aurora.scheduler.base.JobKeys; +import org.apache.aurora.scheduler.base.Jobs; import org.apache.aurora.scheduler.base.Query; import org.apache.aurora.scheduler.base.ScheduleException; import org.apache.aurora.scheduler.base.Tasks; @@ -393,35 +396,39 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { summaries.add(summary); } - return new Response() - .setResponseCode(OK) - .setResult(Result.roleSummaryResult(new RoleSummaryResult(summaries))); - } - - private static Multimap mapByRole( - Iterable tasks, - Function keyExtractor) { - - return HashMultimap.create( - Multimaps.index(Iterables.transform(tasks, keyExtractor), JobKeys.TO_ROLE)); + return okResponse(Result.roleSummaryResult(new RoleSummaryResult(summaries))); } @Override - public Response getJobs(@Nullable String maybeNullRole) { + public Response getJobSummary(@Nullable String maybeNullRole) { Optional ownerRole = Optional.fromNullable(maybeNullRole); + final Multimap tasks = getTasks(ownerRole); + final Map jobs = getJobs(ownerRole, tasks); - // Ensure we only return one JobConfiguration for each JobKey. - Map jobs = Maps.newHashMap(); + Function makeJobSummary = new Function() { + @Override + public JobSummary apply(IJobKey jobKey) { + return new JobSummary() + .setJob(jobs.get(jobKey).newBuilder()) + .setStats(Jobs.getJobStats(tasks.get(jobKey))); + } + }; - // Query the task store, find immediate jobs, and synthesize a JobConfiguration for them. - // This is necessary because the ImmediateJobManager doesn't store jobs directly and - // ImmediateJobManager#getJobs always returns an empty Collection. - Query.Builder scope = ownerRole.isPresent() - ? Query.roleScoped(ownerRole.get()) - : Query.unscoped(); - Multimap tasks = - Tasks.byJobKey(Storage.Util.weaklyConsistentFetchTasks(storage, scope.active())); + ImmutableSet jobSummaries = + FluentIterable.from(jobs.keySet()).transform(makeJobSummary).toSet(); + + return okResponse(Result.jobSummaryResult(new JobSummaryResult().setSummaries(jobSummaries))); + } + + private Map getJobs( + Optional ownerRole, + Multimap tasks) { + + // We need to synthesize the JobConfiguration from the the current tasks because the + // ImmediateJobManager doesn't store jobs directly and ImmediateJobManager#getJobs always + // returns an empty Collection. + Map jobs = Maps.newHashMap(); jobs.putAll(Maps.transformEntries(tasks.asMap(), new Maps.EntryTransformer, IJobConfiguration>() { @@ -430,14 +437,15 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { IJobKey jobKey, Collection tasks) { - // Pick an arbitrary task for each immediate job. The chosen task might not be the most - // recent if the job is in the middle of an update or some shards have been selectively - // created. - TaskConfig firstTask = tasks.iterator().next().getAssignedTask().getTask().newBuilder(); + // Pick the latest transitioned task for each immediate job since the job can be in the + // middle of an update or some shards have been selectively created. + TaskConfig mostRecentTaskConfig = + Tasks.getLatestActiveTask(tasks).getAssignedTask().getTask().newBuilder(); + return IJobConfiguration.build(new JobConfiguration() .setKey(jobKey.newBuilder()) - .setOwner(firstTask.getOwner()) - .setTaskConfig(firstTask) + .setOwner(mostRecentTaskConfig.getOwner()) + .setTaskConfig(mostRecentTaskConfig) .setInstanceCount(tasks.size())); } })); @@ -449,13 +457,35 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { ? Predicates.compose(Predicates.equalTo(ownerRole.get()), JobKeys.CONFIG_TO_ROLE) : Predicates.alwaysTrue(); jobs.putAll(Maps.uniqueIndex( - FluentIterable.from(cronJobManager.getJobs()).filter(configFilter), - JobKeys.FROM_CONFIG)); + FluentIterable.from(cronJobManager.getJobs()).filter(configFilter), + JobKeys.FROM_CONFIG)); + + return jobs; + } - return new Response() - .setResponseCode(OK) - .setResult(Result.getJobsResult(new GetJobsResult() - .setConfigs(IJobConfiguration.toBuildersSet(jobs.values())))); + private Multimap getTasks(Optional ownerRole) { + Query.Builder scope = ownerRole.isPresent() + ? Query.roleScoped(ownerRole.get()) + : Query.unscoped(); + return Tasks.byJobKey(Storage.Util.weaklyConsistentFetchTasks(storage, scope.active())); + } + + private static Multimap mapByRole( + Iterable tasks, + Function keyExtractor) { + + return HashMultimap.create( + Multimaps.index(Iterables.transform(tasks, keyExtractor), JobKeys.TO_ROLE)); + } + + @Override + public Response getJobs(@Nullable String maybeNullRole) { + Optional ownerRole = Optional.fromNullable(maybeNullRole); + + return okResponse(Result.getJobsResult( + new GetJobsResult() + .setConfigs(IJobConfiguration.toBuildersSet( + getJobs(ownerRole, getTasks(ownerRole)).values())))); } private void validateLockForTasks(Optional lock, Iterable tasks) @@ -604,7 +634,7 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { .setProdConsumption(quotaInfo.prodConsumption().newBuilder()) .setNonProdConsumption(quotaInfo.nonProdConsumption().newBuilder()); - return new Response().setResponseCode(OK).setResult(Result.getQuotaResult(result)); + return okResponse(Result.getQuotaResult(result)); } @@ -630,33 +660,27 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { @Override public Response startMaintenance(Hosts hosts, SessionKey session) { - return new Response() - .setResponseCode(OK) - .setResult(Result.startMaintenanceResult(new StartMaintenanceResult() + return okResponse(Result.startMaintenanceResult( + new StartMaintenanceResult() .setStatuses(maintenance.startMaintenance(hosts.getHostNames())))); } @Override public Response drainHosts(Hosts hosts, SessionKey session) { - return new Response() - .setResponseCode(OK) - .setResult(Result.drainHostsResult(new DrainHostsResult() - .setStatuses(maintenance.drain(hosts.getHostNames())))); + return okResponse(Result.drainHostsResult( + new DrainHostsResult().setStatuses(maintenance.drain(hosts.getHostNames())))); } @Override public Response maintenanceStatus(Hosts hosts, SessionKey session) { - return new Response() - .setResponseCode(OK) - .setResult(Result.maintenanceStatusResult(new MaintenanceStatusResult() - .setStatuses(maintenance.getStatus(hosts.getHostNames())))); + return okResponse(Result.maintenanceStatusResult( + new MaintenanceStatusResult().setStatuses(maintenance.getStatus(hosts.getHostNames())))); } @Override public Response endMaintenance(Hosts hosts, SessionKey session) { - return new Response() - .setResponseCode(OK) - .setResult(Result.endMaintenanceResult(new EndMaintenanceResult() + return okResponse(Result.endMaintenanceResult( + new EndMaintenanceResult() .setStatuses(maintenance.endMaintenance(hosts.getHostNames())))); } @@ -692,10 +716,8 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { @Override public Response listBackups(SessionKey session) { - return new Response() - .setResponseCode(OK) - .setResult(Result.listBackupsResult(new ListBackupsResult() - .setBackups(recovery.listBackups()))); + return okResponse(Result.listBackupsResult(new ListBackupsResult() + .setBackups(recovery.listBackups()))); } @Override @@ -897,9 +919,7 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { @Override public Response getVersion() { - return new Response() - .setResponseCode(OK) - .setResult(Result.getVersionResult(CURRENT_API_VERSION)); + return okResponse(Result.getVersionResult(CURRENT_API_VERSION)); } @Override @@ -1004,4 +1024,8 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { static Optional transitionMessage(String user) { return Optional.of("Transition forced by " + user); } + + private static Response okResponse(Result result) { + return new Response().setResponseCode(OK).setResult(result); + } } http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/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 d72b28c..5c5142c 100644 --- a/src/main/thrift/org/apache/aurora/gen/api.thrift +++ b/src/main/thrift/org/apache/aurora/gen/api.thrift @@ -204,6 +204,18 @@ struct JobConfiguration { // [0, instances). } +struct JobStats { + 1: i32 activeTaskCount // Number of tasks in active state for this job. + 2: i32 finishedTaskCount // Number of tasks in finished state for this job. + 3: i32 failedTaskCount // Number of failed tasks for this job. + 4: i32 pendingTaskCount // Number of tasks in pending state for this job. +} + +struct JobSummary { + 1: JobConfiguration job + 2: JobStats stats +} + // A request to add the following instances to an existing job. Used by addInstances. struct AddInstancesConfig { 1: JobKey key @@ -410,6 +422,10 @@ struct RoleSummaryResult { 1: set summaries } +struct JobSummaryResult { + 1: set summaries +} + // meta-data about the thrift server that is wrapped around every thrift response struct ServerInfo { 1: string clusterName @@ -430,6 +446,7 @@ union Result { 15: APIVersion getVersionResult 16: AcquireLockResult acquireLockResult 17: RoleSummaryResult roleSummaryResult + 18: JobSummaryResult jobSummaryResult } struct Response { @@ -447,6 +464,9 @@ service ReadOnlyScheduler { // Returns a summary of the jobs grouped by role. Response getRoleSummary() + // Returns a summary of jobs, optionally only those owned by a specific role. + Response getJobSummary(1: string role) + // Fetches the status of tasks. Response getTasksStatus(1: TaskQuery query) @@ -461,6 +481,10 @@ service ReadOnlyScheduler { // NOTE: This API is deprecated. // Returns the current version of the API implementation Response getVersion() + + // Populates fields in a job configuration as though it were about to be run. + // This can be used to diff a configuration running tasks. + Response populateJobConfig(1: JobConfiguration description) } // Due to assumptions in the client all authenticated RPCs must have a SessionKey as their @@ -471,10 +495,6 @@ service AuroraSchedulerManager extends ReadOnlyScheduler { // name already exists in the cluster. Response createJob(1: JobConfiguration description, 3: Lock lock, 2: SessionKey session) - // Populates fields in a job configuration as though it were about to be run. - // This can be used to diff a configuration running tasks. - Response populateJobConfig(1: JobConfiguration description) - // 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) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java b/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java new file mode 100644 index 0000000..6524e24 --- /dev/null +++ b/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java @@ -0,0 +1,55 @@ +/** + * Copyright 2014 Apache Software Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.aurora.scheduler.base; + +import java.util.Arrays; + +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; + +import org.apache.aurora.gen.JobStats; +import org.apache.aurora.gen.ScheduleStatus; +import org.apache.aurora.scheduler.storage.entities.IScheduledTask; +import org.junit.Test; + +import static org.apache.aurora.scheduler.base.TaskTestUtil.makeTask; +import static org.junit.Assert.assertEquals; + +public class JobsTest { + + @Test + public void testGetJobStats() { + ImmutableList tasks = + FluentIterable + .from(Sets.immutableEnumSet(Arrays.asList(ScheduleStatus.values()))) + .transform(new Function() { + @Override + public IScheduledTask apply(ScheduleStatus status) { + return makeTask(status, 100); + } + }).toList(); + + JobStats expectedStats = new JobStats() + .setActiveTaskCount(7) + .setFailedTaskCount(3) + .setFinishedTaskCount(2) + .setPendingTaskCount(3); + + assertEquals(expectedStats, Jobs.getJobStats(tasks)); + } +} http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/src/test/java/org/apache/aurora/scheduler/base/TaskTestUtil.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/base/TaskTestUtil.java b/src/test/java/org/apache/aurora/scheduler/base/TaskTestUtil.java new file mode 100644 index 0000000..2006ddb --- /dev/null +++ b/src/test/java/org/apache/aurora/scheduler/base/TaskTestUtil.java @@ -0,0 +1,68 @@ +/** + * Copyright 2014 Apache Software Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.aurora.scheduler.base; + +import java.util.Collections; +import java.util.List; + +import com.google.common.collect.Lists; + +import org.apache.aurora.gen.AssignedTask; +import org.apache.aurora.gen.Identity; +import org.apache.aurora.gen.ScheduleStatus; +import org.apache.aurora.gen.ScheduledTask; +import org.apache.aurora.gen.TaskConfig; +import org.apache.aurora.gen.TaskEvent; +import org.apache.aurora.scheduler.storage.entities.IScheduledTask; + +/** + * Convenience methods for working with tasks. + */ +public final class TaskTestUtil { + + private TaskTestUtil() { + // Utility class. + } + + public static IScheduledTask makeTask(ScheduleStatus status, long startTime) { + return makeTask(status, makeTaskEvents(startTime, 3)); + } + + public static IScheduledTask makeTask(ScheduleStatus status, List taskEvents) { + return IScheduledTask.build(new ScheduledTask() + .setStatus(status) + .setAssignedTask(new AssignedTask() + .setTaskId("task_id") + .setSlaveHost("host1") + .setTask(new TaskConfig() + .setJobName("job_a") + .setOwner(new Identity("role_a", "role_a" + "-user")))) + .setTaskEvents(taskEvents)); + } + + private static List makeTaskEvents(long startTs, int count) { + List taskEvents = Lists.newArrayListWithCapacity(3); + for (int i = 0; i < count; i++) { + taskEvents.add(makeTaskEvent(startTs - (i * 10))); + } + Collections.reverse(taskEvents); + return taskEvents; + } + + private static TaskEvent makeTaskEvent(long ts) { + return new TaskEvent().setTimestamp(ts); + } +} http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/src/test/java/org/apache/aurora/scheduler/base/TasksTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/base/TasksTest.java b/src/test/java/org/apache/aurora/scheduler/base/TasksTest.java new file mode 100644 index 0000000..102fe04 --- /dev/null +++ b/src/test/java/org/apache/aurora/scheduler/base/TasksTest.java @@ -0,0 +1,83 @@ +/** + * Copyright 2014 Apache Software Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.aurora.scheduler.base; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import org.apache.aurora.gen.ScheduleStatus; +import org.apache.aurora.scheduler.storage.entities.IScheduledTask; +import org.junit.Test; + +import static org.apache.aurora.gen.ScheduleStatus.FINISHED; +import static org.apache.aurora.gen.ScheduleStatus.RUNNING; +import static org.apache.aurora.scheduler.base.TaskTestUtil.makeTask; +import static org.apache.aurora.scheduler.base.Tasks.getLatestActiveTask; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class TasksTest { + + @Test + public void testOrderedStatusesForCompleteness() { + // OrderedTaskStatuses should contain all ScheduleStatus values except INIT and UNKNOWN. + assertEquals( + ImmutableSet.copyOf(ScheduleStatus.values()), + ImmutableSet.builder() + .addAll(Tasks.ORDERED_TASK_STATUSES) + .add(ScheduleStatus.INIT) + .add(ScheduleStatus.UNKNOWN) + .build()); + } + + @Test + public void testLatestTransitionedTasks() { + IScheduledTask f1 = makeTask(FINISHED, 100); + IScheduledTask f2 = makeTask(FINISHED, 200); + IScheduledTask f3 = makeTask(FINISHED, 300); + IScheduledTask r1 = makeTask(RUNNING, 400); + IScheduledTask r2 = makeTask(RUNNING, 500); + IScheduledTask r3 = makeTask(RUNNING, 600); + + try { + getLatestActiveTask(ImmutableList.of()); + fail("Should have thrown IllegalArgumentException."); + } catch (IllegalArgumentException e) { + // Expected when called with an empty task list. + } + + assertLatestTask(r1, r1); + assertLatestTask(r2, r1, r2); + assertLatestTask(r2, r2, r1); + assertLatestTask(r3, r2, r1, r3); + assertLatestTask(r3, r3, r2, r1); + + assertLatestTask(f1, f1); + assertLatestTask(f2, f1, f2); + assertLatestTask(f2, f2, f1); + assertLatestTask(f3, f2, f1, f3); + assertLatestTask(f3, f3, f2, f1); + + assertLatestTask(r1, f2, f1, r1); + assertLatestTask(r2, f2, f1, r1, r2); + assertLatestTask(r3, f2, f1, f3, r1, r2, r3); + assertLatestTask(r3, r1, r3, r2, f3, f1, f2); + } + + private void assertLatestTask(IScheduledTask expectedLatest, IScheduledTask... tasks) { + assertEquals(expectedLatest, getLatestActiveTask(ImmutableList.copyOf(tasks))); + } +} http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/src/test/java/org/apache/aurora/scheduler/http/SchedulerzRoleTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/http/SchedulerzRoleTest.java b/src/test/java/org/apache/aurora/scheduler/http/SchedulerzRoleTest.java deleted file mode 100644 index 912be18..0000000 --- a/src/test/java/org/apache/aurora/scheduler/http/SchedulerzRoleTest.java +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Copyright 2013 Apache Software Foundation - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.aurora.scheduler.http; - -import java.util.Collections; -import java.util.List; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import org.apache.aurora.gen.AssignedTask; -import org.apache.aurora.gen.Identity; -import org.apache.aurora.gen.ScheduleStatus; -import org.apache.aurora.gen.ScheduledTask; -import org.apache.aurora.gen.TaskConfig; -import org.apache.aurora.gen.TaskEvent; -import org.apache.aurora.scheduler.storage.entities.IScheduledTask; -import org.junit.Test; - -import static org.apache.aurora.gen.ScheduleStatus.FINISHED; -import static org.apache.aurora.gen.ScheduleStatus.RUNNING; -import static org.apache.aurora.scheduler.http.SchedulerzRole.getFreshestTask; -import static org.junit.Assert.assertEquals; - -public class SchedulerzRoleTest { - - @Test - public void testFreshestTasks() { - final IScheduledTask F1 = makeTask(FINISHED, 100); - final IScheduledTask F2 = makeTask(FINISHED, 200); - final IScheduledTask F3 = makeTask(FINISHED, 300); - final IScheduledTask R1 = makeTask(RUNNING, 400); - final IScheduledTask R2 = makeTask(RUNNING, 500); - final IScheduledTask R3 = makeTask(RUNNING, 600); - - assertEquals(R1, getFreshestTask(ImmutableList.of(R1))); - assertEquals(R2, getFreshestTask(ImmutableList.of(R1, R2))); - assertEquals(R2, getFreshestTask(ImmutableList.of(R2, R1))); - assertEquals(R3, getFreshestTask(ImmutableList.of(R2, R1, R3))); - assertEquals(R3, getFreshestTask(ImmutableList.of(R3, R2, R1))); - - assertEquals(F1, getFreshestTask(ImmutableList.of(F1))); - assertEquals(F2, getFreshestTask(ImmutableList.of(F1, F2))); - assertEquals(F2, getFreshestTask(ImmutableList.of(F2, F1))); - assertEquals(F3, getFreshestTask(ImmutableList.of(F2, F1, F3))); - assertEquals(F3, getFreshestTask(ImmutableList.of(F3, F2, F1))); - - assertEquals(R1, getFreshestTask(ImmutableList.of(F2, F1, R1))); - assertEquals(R2, getFreshestTask(ImmutableList.of(F2, F1, R1, R2))); - assertEquals(R3, getFreshestTask(ImmutableList.of(F2, F1, F3, R1, R2, R3))); - assertEquals(R3, getFreshestTask(ImmutableList.of(R1, R3, R2, F3, F1, F2))); - } - - private IScheduledTask makeTask(ScheduleStatus status, long startTime) { - return makeTask(status, makeTaskEvents(startTime, 3)); - } - - private IScheduledTask makeTask(ScheduleStatus status, List taskEvents) { - return IScheduledTask.build(new ScheduledTask() - .setStatus(status) - .setAssignedTask(new AssignedTask() - .setTaskId("task_id") - .setSlaveHost("host1") - .setTask(new TaskConfig() - .setJobName("job_a") - .setOwner(new Identity("role_a", "role_a" + "-user")))) - .setTaskEvents(taskEvents)); - } - - private List makeTaskEvents(long startTs, int count) { - List taskEvents = Lists.newArrayListWithCapacity(3); - for (int i = 0; i < count; i++) { - taskEvents.add(makeTaskEvent(startTs - (i * 10))); - } - Collections.reverse(taskEvents); - return taskEvents; - } - - private TaskEvent makeTaskEvent(long ts) { - return new TaskEvent().setTimestamp(ts); - } -} http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/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 9712f30..60df209 100644 --- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java +++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; import com.google.inject.AbstractModule; import com.google.inject.Guice; import com.google.inject.Injector; @@ -34,6 +35,7 @@ import org.apache.aurora.auth.CapabilityValidator; import org.apache.aurora.auth.CapabilityValidator.AuditCheck; import org.apache.aurora.auth.CapabilityValidator.Capability; import org.apache.aurora.auth.SessionValidator.AuthFailedException; +import org.apache.aurora.gen.APIVersion; import org.apache.aurora.gen.AddInstancesConfig; import org.apache.aurora.gen.AssignedTask; import org.apache.aurora.gen.AuroraAdmin; @@ -48,12 +50,16 @@ import org.apache.aurora.gen.InstanceKey; import org.apache.aurora.gen.JobConfigRewrite; import org.apache.aurora.gen.JobConfiguration; import org.apache.aurora.gen.JobKey; +import org.apache.aurora.gen.JobStats; +import org.apache.aurora.gen.JobSummary; +import org.apache.aurora.gen.JobSummaryResult; import org.apache.aurora.gen.LimitConstraint; import org.apache.aurora.gen.Lock; import org.apache.aurora.gen.LockKey; import org.apache.aurora.gen.ResourceAggregate; import org.apache.aurora.gen.Response; import org.apache.aurora.gen.ResponseCode; +import org.apache.aurora.gen.Result; import org.apache.aurora.gen.RewriteConfigsRequest; import org.apache.aurora.gen.RoleSummary; import org.apache.aurora.gen.RoleSummaryResult; @@ -108,6 +114,7 @@ import static org.apache.aurora.gen.ResponseCode.LOCK_ERROR; import static org.apache.aurora.gen.ResponseCode.OK; import static org.apache.aurora.gen.ResponseCode.WARNING; import static org.apache.aurora.gen.apiConstants.DEFAULT_ENVIRONMENT; +import static org.apache.aurora.gen.apiConstants.THRIFT_API_VERSION; import static org.apache.aurora.scheduler.configuration.ConfigurationManager.DEDICATED_ATTRIBUTE; import static org.apache.aurora.scheduler.thrift.SchedulerThriftInterface.transitionMessage; import static org.easymock.EasyMock.anyObject; @@ -135,6 +142,9 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { private static final IResourceAggregate CONSUMED = IResourceAggregate.build(new ResourceAggregate(0.0, 0, 0)); + private static final ServerInfo SERVER_INFO = + new ServerInfo().setClusterName("test").setThriftAPIVersion(THRIFT_API_VERSION); + private static final APIVersion API_VERSION = new APIVersion().setMajor(THRIFT_API_VERSION); private StorageTestUtil storageUtil; private SchedulerCore scheduler; @@ -178,8 +188,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { bind(CronJobManager.class).toInstance(cronJobManager); bind(QuotaManager.class).toInstance(quotaManager); bind(AuroraAdmin.Iface.class).to(SchedulerThriftInterface.class); - bind(IServerInfo.class).toInstance(IServerInfo.build( - new ServerInfo().setClusterName("test").setThriftAPIVersion(1))); + bind(IServerInfo.class).toInstance(IServerInfo.build(SERVER_INFO)); } }; Injector injector = Guice.createInjector(testModule, new AopModule()); @@ -942,6 +951,98 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { } @Test + public void testGetJobSummary() throws Exception { + TaskConfig ownedCronJobTask = nonProductionTask() + .setJobName(JobKeys.TO_JOB_NAME.apply(JOB_KEY)) + .setOwner(ROLE_IDENTITY) + .setEnvironment(JobKeys.TO_ENVIRONMENT.apply(JOB_KEY)); + JobConfiguration ownedCronJob = makeJob() + .setCronSchedule("0 * * * *") + .setTaskConfig(ownedCronJobTask); + IScheduledTask ownedCronJobScheduledTask = IScheduledTask.build(new ScheduledTask() + .setAssignedTask(new AssignedTask().setTask(ownedCronJobTask)) + .setStatus(ScheduleStatus.ASSIGNED)); + Identity otherOwner = new Identity("other", "other"); + JobConfiguration unownedCronJob = makeJob() + .setOwner(otherOwner) + .setCronSchedule("0 * * * *") + .setKey(JOB_KEY.newBuilder().setRole("other")) + .setTaskConfig(ownedCronJobTask.deepCopy().setOwner(otherOwner)); + TaskConfig ownedImmediateTaskInfo = defaultTask(false) + .setJobName("immediate") + .setOwner(ROLE_IDENTITY); + Set ownedCronJobOnly = ImmutableSet.of(ownedCronJob); + Set ownedCronJobSummaryOnly = ImmutableSet.of( + new JobSummary().setJob(ownedCronJob).setStats(new JobStats())); + Set ownedCronJobSummaryWithRunningTask = ImmutableSet.of( + new JobSummary().setJob(ownedCronJob).setStats(new JobStats().setActiveTaskCount(1))); + Set unownedCronJobOnly = ImmutableSet.of(unownedCronJob); + Set bothCronJobs = ImmutableSet.of(ownedCronJob, unownedCronJob); + + IScheduledTask ownedImmediateTask = IScheduledTask.build(new ScheduledTask() + .setAssignedTask(new AssignedTask().setTask(ownedImmediateTaskInfo)) + .setStatus(ScheduleStatus.ASSIGNED)); + JobConfiguration ownedImmediateJob = new JobConfiguration() + .setKey(JOB_KEY.newBuilder().setName("immediate")) + .setOwner(ROLE_IDENTITY) + .setInstanceCount(1) + .setTaskConfig(ownedImmediateTaskInfo); + Query.Builder query = Query.roleScoped(ROLE).active(); + + Set ownedImmedieteJobSummaryOnly = ImmutableSet.of( + new JobSummary().setJob(ownedImmediateJob).setStats(new JobStats().setActiveTaskCount(1))); + + expect(cronJobManager.getJobs()).andReturn(IJobConfiguration.setFromBuilders(ownedCronJobOnly)); + storageUtil.expectTaskFetch(query); + + expect(cronJobManager.getJobs()).andReturn(IJobConfiguration.setFromBuilders(bothCronJobs)); + storageUtil.expectTaskFetch(query); + + expect(cronJobManager.getJobs()) + .andReturn(IJobConfiguration.setFromBuilders(unownedCronJobOnly)); + storageUtil.expectTaskFetch(query, ownedImmediateTask); + + expect(cronJobManager.getJobs()).andReturn(ImmutableSet.of()); + storageUtil.expectTaskFetch(query); + + // Handle the case where a cron job has a running task (same JobKey present in both stores). + expect(cronJobManager.getJobs()) + .andReturn(ImmutableList.of(IJobConfiguration.build(ownedCronJob))); + storageUtil.expectTaskFetch(query, ownedCronJobScheduledTask); + + control.replay(); + + assertEquals(jobSummaryResponse(ownedCronJobSummaryOnly), thrift.getJobSummary(ROLE)); + + assertEquals(jobSummaryResponse(ownedCronJobSummaryOnly), thrift.getJobSummary(ROLE)); + + Response jobSummaryResponse = thrift.getJobSummary(ROLE); + assertEquals(jobSummaryResponse(ownedImmedieteJobSummaryOnly), jobSummaryResponse); + assertEquals(ownedImmediateTaskInfo, + Iterables.getOnlyElement( + jobSummaryResponse.getResult().getJobSummaryResult().getSummaries()) + .getJob() + .getTaskConfig()); + + assertEquals(jobSummaryResponse(Sets.newHashSet()), thrift.getJobSummary(ROLE)); + + assertEquals(jobSummaryResponse(ownedCronJobSummaryWithRunningTask), + thrift.getJobSummary(ROLE)); + } + + private Response jobSummaryResponse(Set jobSummaries) { + return okResponse(Result.jobSummaryResult(new JobSummaryResult().setSummaries(jobSummaries))); + } + + private Response okResponse(Result result) { + return new Response() + .setResponseCode(OK) + .setDEPRECATEDversion(API_VERSION) + .setServerInfo(SERVER_INFO) + .setResult(result); + } + + @Test public void testGetJobs() throws Exception { TaskConfig ownedCronJobTask = nonProductionTask() .setJobName(JobKeys.TO_JOB_NAME.apply(JOB_KEY)) @@ -951,7 +1052,8 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { .setCronSchedule("0 * * * *") .setTaskConfig(ownedCronJobTask); IScheduledTask ownedCronJobScheduledTask = IScheduledTask.build(new ScheduledTask() - .setAssignedTask(new AssignedTask().setTask(ownedCronJobTask))); + .setAssignedTask(new AssignedTask().setTask(ownedCronJobTask)) + .setStatus(ScheduleStatus.ASSIGNED)); Identity otherOwner = new Identity("other", "other"); JobConfiguration unownedCronJob = makeJob() .setOwner(otherOwner) @@ -965,8 +1067,8 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { Set unownedCronJobOnly = ImmutableSet.of(unownedCronJob); Set bothCronJobs = ImmutableSet.of(ownedCronJob, unownedCronJob); IScheduledTask ownedImmediateTask = IScheduledTask.build(new ScheduledTask() - .setAssignedTask( - new AssignedTask().setTask(ownedImmediateTaskInfo))); + .setAssignedTask(new AssignedTask().setTask(ownedImmediateTaskInfo)) + .setStatus(ScheduleStatus.ASSIGNED)); JobConfiguration ownedImmediateJob = new JobConfiguration() .setKey(JOB_KEY.newBuilder().setName("immediate")) .setOwner(ROLE_IDENTITY) @@ -1027,8 +1129,8 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { .setJobName("immediate") .setOwner(ROLE_IDENTITY); IScheduledTask immediateTask = IScheduledTask.build(new ScheduledTask() - .setAssignedTask( - new AssignedTask().setTask(immediateTaskConfig))); + .setAssignedTask(new AssignedTask().setTask(immediateTaskConfig)) + .setStatus(ScheduleStatus.ASSIGNED)); JobConfiguration immediateJob = new JobConfiguration() .setKey(JOB_KEY.newBuilder().setName("immediate")) .setOwner(ROLE_IDENTITY) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/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 a5fcbd4..933a56b 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 @@ -112,6 +112,11 @@ abstract class ForwardingThrift implements AuroraAdmin.Iface { } @Override + public Response getJobSummary(String role) throws TException { + return delegate.getJobSummary(role); + } + + @Override public Response createJob(JobConfiguration description, Lock lock, SessionKey session) throws TException { http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d96d4bbf/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 2308ba8..250dd6b 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 @@ -0025936691573a8638d5b1a0323c3924 +0f9ef34166894f7e8099b1c355bb81f9