diff --git a/.classpath b/.classpath index 96565d6..11d2795 100644 --- a/.classpath +++ b/.classpath @@ -27,6 +27,7 @@ + diff --git a/azkaban-common/src/java/azkaban/common/jobs/AbstractJob.java b/azkaban-common/src/java/azkaban/common/jobs/AbstractJob.java index d76d0e1..71cb3ba 100644 --- a/azkaban-common/src/java/azkaban/common/jobs/AbstractJob.java +++ b/azkaban-common/src/java/azkaban/common/jobs/AbstractJob.java @@ -93,5 +93,9 @@ public Props getJobGeneratedProperties() { } public abstract void run() throws Exception; + + public boolean isCanceled() { + return false; + } } diff --git a/azkaban-common/src/java/azkaban/common/jobs/Job.java b/azkaban-common/src/java/azkaban/common/jobs/Job.java index cb2fdde..ec644ce 100644 --- a/azkaban-common/src/java/azkaban/common/jobs/Job.java +++ b/azkaban-common/src/java/azkaban/common/jobs/Job.java @@ -66,4 +66,10 @@ public interface Job { * @return */ public Props getJobGeneratedProperties(); + + /** + * Determine if the job was cancelled. + * @return + */ + public boolean isCanceled(); } diff --git a/azkaban/src/java/azkaban/app/AzkabanApplication.java b/azkaban/src/java/azkaban/app/AzkabanApplication.java index e3f2495..4ae225f 100644 --- a/azkaban/src/java/azkaban/app/AzkabanApplication.java +++ b/azkaban/src/java/azkaban/app/AzkabanApplication.java @@ -49,6 +49,9 @@ import azkaban.jobs.builtin.PythonJob; import azkaban.jobs.builtin.RubyJob; import azkaban.jobs.builtin.ScriptJob; +import azkaban.monitor.MonitorImpl; +import azkaban.monitor.MonitorInterface; +import azkaban.monitor.MonitorInternalInterface; import azkaban.scheduler.LocalFileScheduleLoader; import azkaban.scheduler.ScheduleManager; import azkaban.serialization.DefaultExecutableFlowSerializer; @@ -85,6 +88,7 @@ public class AzkabanApplication private final ClassLoader _baseClassLoader; private final String _hdfsUrl; private final FlowManager _allFlows; + private final MonitorImpl _monitor; private final JobExecutorManager _jobExecutorManager; private final ScheduleManager _schedulerManager; @@ -169,6 +173,8 @@ public AzkabanApplication(List jobDirs, File logDir, File tempDir, boolean FlowExecutionSerializer flowExecutionSerializer = new FlowExecutionSerializer(flowSerializer); FlowExecutionDeserializer flowExecutionDeserializer = new FlowExecutionDeserializer(flowDeserializer); + _monitor = (MonitorImpl)MonitorImpl.getMonitor(); + _allFlows = new CachingFlowManager( new RefreshableFlowManager( _jobManager, @@ -370,4 +376,12 @@ public String getRuntimeProperty(String name) { public void setRuntimeProperty(String key, String value) { _jobExecutorManager.setRuntimeProperty(key, value); } + + public MonitorInterface getMonitor() { + return _monitor; + } + + public MonitorInternalInterface getInternalMonitor() { + return _monitor; + } } diff --git a/azkaban/src/java/azkaban/app/LoggingJob.java b/azkaban/src/java/azkaban/app/LoggingJob.java index 2f19145..7234b0c 100644 --- a/azkaban/src/java/azkaban/app/LoggingJob.java +++ b/azkaban/src/java/azkaban/app/LoggingJob.java @@ -31,6 +31,9 @@ import azkaban.common.jobs.Job; import azkaban.common.utils.Props; import azkaban.common.utils.Utils; +import azkaban.monitor.MonitorImpl; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInternalInterface.JobAction; /** * A wrapper for a job that attaches a Log4J appender to write to the logs @@ -52,6 +55,10 @@ public LoggingJob(String logDir, Job innerJob, String loggerName) { this._logDir = Utils.nonNull(logDir); this._logger = Logger.getLogger(loggerName); } + + public synchronized boolean isCanceled() { + return getInnerJob().isCanceled(); + } public LoggingJob(String logDir, Job innerJob, String loggerName, String loggerPattern) { super(innerJob); @@ -90,10 +97,27 @@ public void run() { boolean jobNotStaleException = false; long start = System.currentTimeMillis(); try { + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.START_WORKFLOW_JOB, + JobState.NOP); + getInnerJob().run(); succeeded = true; + + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + JobState.SUCCESSFUL); } catch(Exception e) { _logger.error("Fatal error occurred while running job '" + jobName + "':", e); + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + getInnerJob().isCanceled() ? JobState.CANCELED : JobState.FAILED); if(e instanceof RuntimeException) throw (RuntimeException) e; else diff --git a/azkaban/src/java/azkaban/flow/IndividualJobExecutableFlow.java b/azkaban/src/java/azkaban/flow/IndividualJobExecutableFlow.java index 0dfb369..150b19e 100644 --- a/azkaban/src/java/azkaban/flow/IndividualJobExecutableFlow.java +++ b/azkaban/src/java/azkaban/flow/IndividualJobExecutableFlow.java @@ -30,6 +30,9 @@ import azkaban.common.jobs.Job; import azkaban.common.utils.Props; import azkaban.jobs.Status; +import azkaban.monitor.MonitorImpl; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInternalInterface.JobAction; /** * An implemention of the ExecutableFlow interface that just @@ -166,6 +169,12 @@ else if (jobState != Status.COMPLETED && ! this.parentProps.equalsProps(parentPr public void run() { final List callbackList; + + MonitorImpl.getInternalMonitorInterface().jobEvent( + job, + System.currentTimeMillis(), + JobAction.START_WORKFLOW_JOB, + JobState.NOP); try { job.run(); @@ -177,6 +186,13 @@ public void run() exceptions.put(getName(), e); callbackList = callbacksToCall; // Get the reference before leaving the synchronized } + + MonitorImpl.getInternalMonitorInterface().jobEvent( + job, + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + JobState.FAILED); + callCallbacks(callbackList, jobState); throw new RuntimeException(e); @@ -188,6 +204,11 @@ public void run() callbackList = callbacksToCall; // Get the reference before leaving the synchronized } + MonitorImpl.getInternalMonitorInterface().jobEvent( + job, + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + JobState.SUCCESSFUL); returnProps.logProperties(String.format("Return props for job[%s]", getName())); callCallbacks(callbackList, jobState); diff --git a/azkaban/src/java/azkaban/jobcontrol/impl/jobs/ResourceThrottledJob.java b/azkaban/src/java/azkaban/jobcontrol/impl/jobs/ResourceThrottledJob.java index 0e080f2..6765579 100644 --- a/azkaban/src/java/azkaban/jobcontrol/impl/jobs/ResourceThrottledJob.java +++ b/azkaban/src/java/azkaban/jobcontrol/impl/jobs/ResourceThrottledJob.java @@ -19,6 +19,9 @@ import org.apache.log4j.Logger; import azkaban.jobcontrol.impl.jobs.locks.JobLock; +import azkaban.monitor.MonitorImpl; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInternalInterface.JobAction; import azkaban.common.jobs.DelegatingJob; import azkaban.common.jobs.Job; @@ -60,17 +63,45 @@ public void run() throws Exception } long totalWait = System.currentTimeMillis() - start; _logger.info(_jobLock + " Time: " + totalWait + " ms."); + + MonitorImpl.getInternalMonitorInterface().workflowResourceThrottledJobEvent(this, totalWait); + try { boolean shouldRunJob; synchronized(lock) { shouldRunJob = ! canceled; } + + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.START_WORKFLOW_JOB, + JobState.NOP); + if(shouldRunJob) { getInnerJob().run(); + + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + JobState.SUCCESSFUL); } else { _logger.info("Job was canceled while waiting for lock. Not running."); + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + JobState.CANCELED); } + } catch (Exception e) { + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + JobState.FAILED); + throw e; } finally { _jobLock.releaseLock(); } @@ -85,4 +116,8 @@ public void cancel() throws Exception super.cancel(); } } + + public synchronized boolean isCanceled() { + return canceled; + } } diff --git a/azkaban/src/java/azkaban/jobcontrol/impl/jobs/RetryingJob.java b/azkaban/src/java/azkaban/jobcontrol/impl/jobs/RetryingJob.java index 1d596c3..abd3310 100644 --- a/azkaban/src/java/azkaban/jobcontrol/impl/jobs/RetryingJob.java +++ b/azkaban/src/java/azkaban/jobcontrol/impl/jobs/RetryingJob.java @@ -21,6 +21,9 @@ import azkaban.common.jobs.DelegatingJob; import azkaban.common.jobs.Job; import azkaban.common.jobs.JobFailedException; +import azkaban.monitor.MonitorImpl; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInternalInterface.JobAction; public class RetryingJob extends DelegatingJob { @@ -49,12 +52,23 @@ public void run() { return; } } - _logger.info("Retrying failed job '" + getInnerJob().getId() + " for attempt " - + (tries + 1)); + _logger.info("Retrying failed job '" + getInnerJob().getId() + " for attempt " + (tries + 1)); } + + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.START_WORKFLOW_JOB, + JobState.NOP); try { getInnerJob().run(); + + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + JobState.SUCCESSFUL); return; } catch(Exception e) { _logger.error("Job '" + getInnerJob().getId() + " failed attempt " + (tries + 1), e); @@ -62,13 +76,23 @@ public void run() { for(int i = 0; i < tries + 1; i++) sadness += ":-( "; _logger.info(sadness); + + MonitorImpl.getInternalMonitorInterface().jobEvent( + getInnerJob(), + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, + getInnerJob().isCanceled() ? JobState.CANCELED : JobState.FAILED); } } // if we get here it means we haven't succeded (otherwise we would have // returned) - throw new JobFailedException(_retries + " run attempt" + (_retries > 1 ? "s" : "") - + " failed."); + throw new JobFailedException(_retries + " run attempt" + (_retries > 1 ? "s" : "") + + " failed."); + } + + public synchronized boolean isCanceled() { + return getInnerJob().isCanceled(); } } diff --git a/azkaban/src/java/azkaban/jobs/JobExecutorManager.java b/azkaban/src/java/azkaban/jobs/JobExecutorManager.java index 075e15f..e516b7d 100644 --- a/azkaban/src/java/azkaban/jobs/JobExecutorManager.java +++ b/azkaban/src/java/azkaban/jobs/JobExecutorManager.java @@ -37,6 +37,9 @@ import azkaban.flow.FlowCallback; import azkaban.flow.FlowExecutionHolder; import azkaban.flow.FlowManager; +import azkaban.monitor.MonitorImpl; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.MonitorInternalInterface.WorkflowAction; import azkaban.util.process.ProcessFailureException; public class JobExecutorManager { @@ -126,6 +129,11 @@ public void execute(ExecutableFlow flow) { final JobExecution executingJob = new JobExecution(flow.getName(), new DateTime(), true); + MonitorImpl.getInternalMonitorInterface().workflowEvent(flow.getId(), + System.currentTimeMillis(), + WorkflowAction.START_WORKFLOW, + WorkflowState.NOP, + flow.getName()); executor.execute(new ExecutingFlowRunnable(holder, executingJob)); } @@ -405,6 +413,13 @@ public void progressMade() { public void completed(Status status) { runningJob.setEndTime(new DateTime()); + MonitorImpl.getInternalMonitorInterface().workflowEvent(flow.getId(), + System.currentTimeMillis(), + WorkflowAction.END_WORKFLOW, + (status == Status.SUCCEEDED ? WorkflowState.SUCCESSFUL : + (status == Status.FAILED ? WorkflowState.FAILED : WorkflowState.UNKNOWN)), + flow.getName()); + try { allKnownFlows.saveExecutableFlow(holder); switch(status) { diff --git a/azkaban/src/java/azkaban/jobs/builtin/NoopJob.java b/azkaban/src/java/azkaban/jobs/builtin/NoopJob.java index aa65359..8bade02 100644 --- a/azkaban/src/java/azkaban/jobs/builtin/NoopJob.java +++ b/azkaban/src/java/azkaban/jobs/builtin/NoopJob.java @@ -56,4 +56,9 @@ public Props getJobGeneratedProperties() { return new Props(); } + + @Override + public boolean isCanceled() { + return false; + } } diff --git a/azkaban/src/java/azkaban/jobs/builtin/ProcessJob.java b/azkaban/src/java/azkaban/jobs/builtin/ProcessJob.java index 0658c50..4371608 100644 --- a/azkaban/src/java/azkaban/jobs/builtin/ProcessJob.java +++ b/azkaban/src/java/azkaban/jobs/builtin/ProcessJob.java @@ -43,14 +43,17 @@ public class ProcessJob extends AbstractProcessJob implements Job { private volatile Process _process; private volatile boolean _isComplete; - + private volatile boolean _isCancelled; + public ProcessJob(JobDescriptor descriptor) { super(descriptor); } public void run() { - + synchronized(this) { + _isCancelled = false; + } resolveProps(); // Sets a list of all the commands that need to be run. @@ -157,6 +160,9 @@ public void cancel() throws Exception { warn("Force kill the process"); _process.destroy(); } + synchronized (this) { + _isCancelled = true; + } } } @@ -314,5 +320,9 @@ public static String[] partitionCommandLine(String command) { return commands.toArray(new String[commands.size()]); } + + public synchronized boolean isCanceled() { + return _isCancelled; + } } diff --git a/azkaban/src/java/azkaban/monitor/MonitorException.java b/azkaban/src/java/azkaban/monitor/MonitorException.java new file mode 100644 index 0000000..885d007 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/MonitorException.java @@ -0,0 +1,54 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +/** + * Exception class specialized to errors in the monitoring + * sub-system of Azkaban. + * + */ +public class MonitorException extends RuntimeException { + + private final static long serialVersionUID = 1; + + /** + * Constructor. + * @param message + * A specific message about this exception. + */ + public MonitorException(String message) { + super(message); + } + + /** + * Constructor. + * @param cause + * A Throwable serving as the cause of this exception. + */ + public MonitorException(Throwable cause) { + super(cause); + } + + /** + * Constructor. + * @param message + * A specific message about this exception. + * @param cause + * A Throwable serving as the cause of this exception. + */ + public MonitorException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/azkaban/src/java/azkaban/monitor/MonitorImpl.java b/azkaban/src/java/azkaban/monitor/MonitorImpl.java new file mode 100644 index 0000000..ee748f3 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/MonitorImpl.java @@ -0,0 +1,648 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Map.Entry; + +import org.apache.log4j.Logger; + +import azkaban.common.jobs.Job; +import azkaban.jobcontrol.impl.jobs.ResourceThrottledJob; +import azkaban.monitor.model.ExecutionModel; +import azkaban.monitor.model.ExecutionModelImpl; +import azkaban.monitor.model.WorkflowExecutionModel; +import azkaban.monitor.stats.ClassStats; +import azkaban.monitor.stats.NativeGlobalStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +/** + * Implements both: + * 1) AzMonitorInterface for clients to use for statistics and notification + * 2) AzMonitorInternalInterface for fielding events from within Azkaban + * + * This object then implements interfaces for: + * 1) Running the Azkaban workflow and job events through the execution model. + * 2) Accessors to global, workflow class, job class statistics + * 3) Methods for managing the notifications. + * + */ +public class MonitorImpl implements MonitorInterface, MonitorInternalInterface { + /** + * Logger dedicated to this file. + */ + private static final Logger LOG = Logger.getLogger(MonitorImpl.class); + + /** + * A map over all notifiers to NotificationDetails which tally all response events + * to which the notifier is to react. + */ + private Map notifiers = + new HashMap(); + + /** + * Maps workflow root name to list of notifiers that want the workflow class + * statistics for said workflow root type. This enumeration shortens the + * notification run time. + */ + private Map> workflowNotifiers = + new HashMap>(); + + /** + * Maps job name to list of notifiers that want the job class statistics + * for said job type. This enumeration shortens the notification run + * time. + */ + private Map> jobNotifiers = + new HashMap>(); + + /** + * The model for all workflow and job instances. + */ + private ExecutionModel eModel; + + /** + * The monitor is a singleton, with a condition. + * The condition is that AzkabanApplication constructs the monitor. It is + * the right place of control to decide when to construct it. + * All other accesses are done through the static getMonitor() method. + * By having the monitor as a singleton, it allows many places in the Azkaban code + * to get the monitor without laboreously passing the monitor around the code. + */ + private static MonitorImpl monitor; + + /** + * Constructor + * Note: This should only be called by AzkabanApplication. + * A public constructor is a little atypical for a singleton pattern. + * What is wanted here is for a static accessor 'getMonitor' to be used throughout + * the Azkaban system. However, the object construction has to be done at the + * right place. + */ + private MonitorImpl() { + eModel = new ExecutionModelImpl(this); + } + + /** + * General accessor to the monitor. + * Note: This should be called first by AzkabanApplication. + * @return the MonitorImpl + */ + public static MonitorImpl getMonitor() { + synchronized (MonitorImpl.class) { + if (monitor == null) { + monitor = new MonitorImpl(); + } + } + return monitor; + } + + /** + * Get rid of the monitor; used for testing. + */ + public static void unsetMonitor() { + synchronized (MonitorImpl.class) { + monitor = null; + } + } + + /** + * getter. + * @return the MonitorInterface interface. + */ + public static MonitorInterface getMonitorInterface() { + return getMonitor(); + } + + /** + * getter + * @return the MonitorInternalInterface interface. + */ + public static MonitorInternalInterface getInternalMonitorInterface() { + return getMonitor(); + } + + /**************** Global Stats accessors (MonitorInterface) ****************/ + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getGlobalAzkabanStats + * + */ + public synchronized NativeGlobalStats getGlobalAzkabanStats() { + return eModel.getGlobalStatsCopy(); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getWorkflowClassRootJobNames + * + */ + public synchronized List getWorkflowClassRootJobNames() { + return eModel.getWorkflowClassIds(); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getWorkflowClassStatsByRootJobName + * + */ + public synchronized NativeWorkflowClassStats getWorkflowClassStatsByRootJobName(final String classId) { + return eModel.getWorkflowClassStatsById(classId); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getAllWorkflowClassStats + * + */ + public synchronized Map getAllWorkflowClassStats() { + return eModel.getAllWorkflowClassStats(); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getJobClassNames + * + */ + public synchronized List getJobClassNames() { + return eModel.getJobClassIds(); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getJobClassStatsByName + * + */ + public synchronized NativeJobClassStats getJobClassStatsByName(final String jobClassId) { + return eModel.getJobClassStatsById(jobClassId); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getAllJobClassStats + * + */ + public synchronized Map getAllJobClassStats() { + return eModel.getAllJobClassStats(); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getCompletedWorkflowModels + * + */ + public synchronized List getCompletedWorkflowModels( + final long bufferTimeMS, + final boolean reverseTime) { + return eModel.getCompletedWorkflowModels(bufferTimeMS, reverseTime); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getNumberOfWorkflows + * + */ + public synchronized long getNumberOfWorkflows() { + return eModel.getNumberOfWorkflows(); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getNumberOfCompletedWorkflows + * + */ + public synchronized long getNumberOfCompletedWorkflows() { + return eModel.getNumberOfCompletedWorkflows(); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#getCompletedWorkflowModels + * + */ + public synchronized List getCompletedWorkflowModels(final boolean reverseTime) { + return eModel.getCompletedWorkflowModels(reverseTime); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#clearCompletedWorkflows + * + */ + public synchronized void clearCompletedWorkflows(final long bufferTimeMS) { + clearCompletedWorkflows(bufferTimeMS); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#clearCompletedWorkflows + * + */ + public void clearCompletedWorkflows() { + eModel.clearCompletedWorkflows(); + } + + /*************** Event interface (AzMonitorInternalInterface) ***************/ + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInternalInterface#workflowEvent + * + */ + public synchronized void workflowEvent(final String wfId, + final long time, + final WorkflowAction action, + final WorkflowState wfState, + final String rootJobName) { + switch (action) { + case SCHEDULE_WORKFLOW: + LOG.debug("Schedule workflow: " + rootJobName); + eModel.scheduleWorkflow(rootJobName, time); + break; + case UNSCHEDULE_WORKFLOW: + LOG.debug("Unscheduled workflow: " + rootJobName); + // TODO + break; + case START_WORKFLOW: + LOG.debug("Start workflow: " + wfId); + eModel.startWorkflow(rootJobName, wfId, time); + break; + case END_WORKFLOW: + if (wfState == WorkflowState.NOP) { + throw new MonitorException("Nop cannot be used as a job state for EndWorkflow."); + } + eModel.endWorkflow(wfId, time, wfState); + break; + default: + LOG.error("Illegal or unknown WorkflowAction specified."); + break; + } + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInternalInterface#jobEvent + * + */ + public synchronized void jobEvent(final Job job, + final long time, + final JobAction action, + final JobState jobState) { + switch (action) { + case START_WORKFLOW_JOB: + LOG.debug("Start workflow job: " + job.getId()); + eModel.startWorkflowJob(time, job); + break; + case END_WORKFLOW_JOB: + LOG.debug("End workflow job: " + job.getId()); + if (jobState == JobState.NOP) { + throw new MonitorException("Nop cannot be used as a job state for EndWorkflowJob."); + } + eModel.endWorkflowJob(time, job, jobState); + break; + default: + LOG.error("Unknown or illegal JobAction specified."); + break; + } + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInternalInterface#workflowResourceThrottledJobEvent + * + */ + public synchronized void workflowResourceThrottledJobEvent(final ResourceThrottledJob job, + final long lockWaitTime) { + eModel.resourceWaitTime(lockWaitTime, job); + } + + //*************** Event notification registration (AzMonitorInterface) ***************// + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#registerGlobalNotification + * + */ + public synchronized void registerGlobalNotification(final MonitorListener notifier, + final GlobalNotificationType type) { + NotificationDetails details = getNotifierDetails(notifier); + switch(type) { + case GLOBAL_STATS_CHANGE: + details.setGlobalStatsMonitor(true); + break; + case ANY_WORKFLOW_CLASS_STATS_CHANGE: + details.setAllWorkflowClassStatsMonitor(true); + break; + case ANY_JOB_CLASS_STATS_CHANGE: + details.setAllJobClassStatsMonitor(true); + break; + default: + LOG.error("Illegal or unknown GlobalNotificationType specified"); + break; + } + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#deregisterGlobalNotification + * + */ + public synchronized void deregisterGlobalNotification(final MonitorListener notifier, + final GlobalNotificationType type) { + NotificationDetails details = getNotifierDetails(notifier); + switch(type) { + case GLOBAL_STATS_CHANGE: + details.setGlobalStatsMonitor(false); + break; + case ANY_WORKFLOW_CLASS_STATS_CHANGE: + details.setAllWorkflowClassStatsMonitor(false); + break; + case ANY_JOB_CLASS_STATS_CHANGE: + details.setAllJobClassStatsMonitor(false); + break; + default: + LOG.error("Illegal or unknown GlobalNotificationType specified"); + break; + } + + // See if this notification is serving any purpose, and just remove if not. + checkDoDeregisterNotification(notifier); + } + + /** + * Method to eliminate notifiers that aren't monitoring anything. + * @param notifier + * MonitorListener we are examining. + */ + private void checkDoDeregisterNotification(final MonitorListener notifier) { + NotificationDetails details = getNotifierDetails(notifier); + if (details == null) { + return; + } + + if (!details.hasAllJobClassStatsMonitor() && !details.hasAllJobClassStatsMonitor() + && !details.hasAllWorkflowClassStatsMonitor() + && details.getGlobalWorkflowClasses().size() == 0 + && details.getGlobalJobClasses().size() == 0) { + notifiers.remove(notifier); + } + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#deregisterNotifications + * + */ + public synchronized void deregisterNotifications(final MonitorListener notifier) { + NotificationDetails details = notifiers.get(notifier); + if (details == null) { + LOG.error("Attempt to deregister notification that never registered"); + return; + } + notifiers.remove(notifier); + + Set wfClasses = details.getGlobalWorkflowClasses(); + for (String wfClassName : wfClasses) { + List notifiers = workflowNotifiers.get(wfClassName); + notifiers.remove(notifier); + if (notifiers.size() == 0) { + workflowNotifiers.remove(wfClassName); + } + } + + Set jobClasses = details.getGlobalJobClasses(); + for (String jobClassName : jobClasses) { + List notifiers = jobNotifiers.get(jobClassName); + notifiers.remove(notifier); + if (notifiers.size() == 0) { + jobNotifiers.remove(jobClassName); + } + } + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#registerWorkflowClassNotification + * + */ + public synchronized void registerWorkflowClassNotification(final MonitorListener notifier, + final String workflowClassId) { + registerClassNotification(notifier, workflowClassId, workflowNotifiers); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#registerJobClassNotification + * + */ + public synchronized void registerJobClassNotification(final MonitorListener notifier, + final String jobClassId) { + registerClassNotification(notifier, jobClassId, jobNotifiers); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#registerClassNotification + * + */ + private void registerClassNotification(final MonitorListener notifier, + final String classId, + final Map> notifierMap) { + NotificationDetails details = getNotifierDetails(notifier); + + details.getGlobalWorkflowClasses().add(classId); + + + List notifierList = notifierMap.get(classId); + if (notifierList == null) { + notifierList = new ArrayList(); + notifierMap.put(classId, notifierList); + } + + if (!notifierList.contains(notifier)) { + notifierList.add(notifier); + } + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#deregisterWorkflowClassNotification + * + */ + public synchronized void deregisterWorkflowClassNotification(final MonitorListener notifier, + final String workflowClassId) { + deregisterClassNotification(notifier, workflowClassId, workflowNotifiers); + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInterface#deregisterJobClassNotification + * + */ + public synchronized void deregisterJobClassNotification(final MonitorListener notifier, + final String jobClassId) { + deregisterClassNotification(notifier, jobClassId, jobNotifiers); + } + + /** + * Deregister a monitor class notifier. + * @param notifier + * The notifier to deregister. + * @param classId + * The root job id to deregister against. + * @param notifierMap + * the map between classId and MonitorListeners to be adjusted. + */ + private void deregisterClassNotification(final MonitorListener notifier, + final String classId, + final Map> notifierMap) { + NotificationDetails details = getNotifierDetails(notifier); + + details.getGlobalWorkflowClasses().remove(classId); + + List notifiers = notifierMap.get(classId); + notifiers.remove(notifier); + if (notifiers.size() == 0) { + notifierMap.remove(classId); + } + + // See if this notification is serving any purpose, and just remove if not. + checkDoDeregisterNotification(notifier); + } + + /** + * Used in testing. + * @return a list of all registered MonitorNotification's. + */ + protected synchronized List getAllNotifiers() { + return new ArrayList(notifiers.keySet()); + } + + //*************** Event notification (AzMonitorInternalInterface) ***************// + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInternalInterface#executeGlobalNotify + * + */ + public synchronized void executeGlobalNotify(final GlobalNotificationType eventType, + final ClassStats statsObject) { + switch (eventType) { + case GLOBAL_STATS_CHANGE: + for (Entry entry : notifiers.entrySet()) { + if (entry.getValue().hasGlobalStatsMonitor()) { + entry.getKey().onGlobalNotify(eventType, statsObject); + } + } + break; + case ANY_WORKFLOW_CLASS_STATS_CHANGE: + for (Entry entry : notifiers.entrySet()) { + if (entry.getValue().hasAllWorkflowClassStatsMonitor()) { + entry.getKey().onGlobalNotify(eventType, statsObject); + } + } + break; + case ANY_JOB_CLASS_STATS_CHANGE: + for (Entry entry : notifiers.entrySet()) { + if (entry.getValue().hasAllJobClassStatsMonitor()) { + entry.getKey().onGlobalNotify(eventType, statsObject); + } + } + break; + default: + LOG.error("Illegal or unknown GlobalNotificationType specified."); + break; + } + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInternalInterface#executeWorkflowClassNotify + * + */ + public synchronized void executeWorkflowClassNotify(final NativeWorkflowClassStats wfStats) { + List notifiers = + workflowNotifiers.get(wfStats.getWorkflowRootName()); + if (notifiers != null) { + for (MonitorListener notifier : notifiers) { + notifier.onWorkflowNotify(wfStats); + } + } + } + + /** + * {@inheritDoc} + * + * @see azkaban.monitor.MonitorInternalInterface#executeJobClassNotify + * + */ + public synchronized void executeJobClassNotify(final NativeJobClassStats jobStats) { + List notifiers = + jobNotifiers.get(jobStats.getJobClassName()); + if (notifiers != null) { + for (MonitorListener notifier : notifiers) { + notifier.onJobNotify(jobStats); + } + } + } + + /** + * Get details about a given notifier. + * @param notifier + * The MonitorListener whose details are requested. + * @return + * NotificationDetails + */ + private synchronized NotificationDetails getNotifierDetails(final MonitorListener notifier) { + NotificationDetails details = notifiers.get(notifier); + if (details == null) { + details = new NotificationDetails(notifier); + notifiers.put(notifier, details); + } + return details; + } +} diff --git a/azkaban/src/java/azkaban/monitor/MonitorInterface.java b/azkaban/src/java/azkaban/monitor/MonitorInterface.java new file mode 100644 index 0000000..54cd4a6 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/MonitorInterface.java @@ -0,0 +1,272 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import java.util.List; +import java.util.Map; + +import azkaban.monitor.model.WorkflowExecutionModel; +import azkaban.monitor.stats.NativeGlobalStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +/** + * This is the interface to the monitor system that should be used by application clients + * that want to acquire job run statistics. It consists of the following: + * 1) A set of Global, Workflow, and Job state enums + * 2) A set of accessors to Global, Workflow, and Job statistics objects. + * 3) A set of registration notification methods to enable state update notification. + * + */ +public interface MonitorInterface { + + /** + * Job termination states. + */ + public static enum JobState { + /** + * Job successful + */ + SUCCESSFUL, + /** + * Job failed + */ + FAILED, + /** + * Job canceled + */ + CANCELED, + /** + * Job unknown + */ + UNKNOWN, + /** + * Use this value when JobState irrelevant to the event, e.g. start job. + */ + NOP, + } + + /** + * Workflow termination states. + */ + public static enum WorkflowState { + /** + * Workflow successful + */ + SUCCESSFUL, + /** + * Workflow failed + */ + FAILED, + /** + * Workflow canceled + */ + CANCELED, + /** + * Workflow unknown + */ + UNKNOWN, + /** + * Use this value when WorkflowState is irrelevant to the event, e.g. start workflow. + */ + NOP, + } + + /** + * Global notification enablement enums. + */ + public static enum GlobalNotificationType { + /** + * Indicates global stats over server. + */ + GLOBAL_STATS_CHANGE, + /** + * Indicates aggregate workflow stats based on root job name. + */ + ANY_WORKFLOW_CLASS_STATS_CHANGE, + /** + * Indicates aggregate job stats based on job name. + */ + ANY_JOB_CLASS_STATS_CHANGE, + } + + //***** Global Statistics accessor methods *****// + + /** + * Get a copy of the global azkaban statistics. + * @return NativeGlobalStats a copy. + */ + NativeGlobalStats getGlobalAzkabanStats(); + + /** + * Get a list of all the current workflow as id'd about root job name. + * @return List of all job names serving as a root of a workflow. + */ + List getWorkflowClassRootJobNames(); + + /** + * Get a copy of a WorkflowClassStats based on a given root job name. + * @param rootJobName + * the root job name on which we search for the workflow class stats. + * @return NativeWorkflowClassStats a copy if found based on classId. + */ + NativeWorkflowClassStats getWorkflowClassStatsByRootJobName(String rootJobName); + + /** + * Get a copy of all the current workflow class stats objects. + * @return Map + * mapping job root name to workflow class stats. + */ + Map getAllWorkflowClassStats(); + + /** + * Get a list of all the Job Classes available. + * @return List listing all job names whose stats are being collected. + */ + List getJobClassNames(); + + /** + * Get a copy of the JobClassStats as given by the job class name. + * @param jobClassName + * the job id on which we are searching for the job stats + * @return NativeJobClassStats a copy if found. + */ + NativeJobClassStats getJobClassStatsByName(String jobClassName); + + /** + * Get set of all JobClassStats. This will be a full copy, not the originals. + * @return Map mapping the name of the job + * to its NativeJobClassStats. + */ + Map getAllJobClassStats(); + + /** + * Return the number of workflows, active and completed. + * @return the number of workflow instances that are being monitored. + */ + long getNumberOfWorkflows(); + + /** + * Return the number of completed workflows. + * @return long value for the number of completed workflow + * instances that were monitored. + */ + long getNumberOfCompletedWorkflows(); + + /** + * Return a list of all currently completed workflow models. + * @param reverseTime - boolean to indicate reverse order. + * @return List - a list of all workflow model instances + * that completed. + */ + List getCompletedWorkflowModels(boolean reverseTime); + + /** + * Return a list of workflow models completed prior to a given time. + * @param endedBeforeTime + * epoch time before which workflows must end. + * @param reverseTime + * boolean true indicates return list in reverse time order. + * @return + * List + * + */ + List getCompletedWorkflowModels( + long endedBeforeTime, + boolean reverseTime); + + /** + * Clear out all completed workflow models that ended some amount + * of time earlier than current time. + * @param endedBeforeTime + * epoch time before which workflows must end. + */ + void clearCompletedWorkflows(long endedBeforeTime); + + /** + * Clear out all the completed workflow models - used to free storage. + */ + void clearCompletedWorkflows(); + + //***** Notification Methods *****// + + /** + * Register notifier for global stats, or any workflow class or job class changes. + * @param notifier + * notification interface that should be registered. + * @param type + * based on GlobalNotificationType, the type of notification requested. + */ + void registerGlobalNotification(MonitorListener notifier, + GlobalNotificationType type); + + /** + * Deregister notifier for a type of global notification. + * @param notifier + * notification interface that should be deregistered. + * @param type + * based on GlobalNotificationType, the type of notification to deregister. + */ + void deregisterGlobalNotification(MonitorListener notifier, + GlobalNotificationType type); + + + /** + * Register for changes on a particular workflow class, given by root job name. + * @param notifier + * notifier to be registered + * @param workflowRootJobName + * name of the workflow root job on which to be registered. + */ + void registerWorkflowClassNotification(MonitorListener notifier, + String workflowRootJobName); + + /** + * Deregister a notifier for notifications on a workflow class. + * @param notifier + * notifier to be deregistered. + * @param workflowRootJobName + * name of workflow root job on which to deregister. + */ + void deregisterWorkflowClassNotification(MonitorListener notifier, + String workflowRootJobName); + + /** + * Register for changes on a particular job class - as indicated by job name. + * @param notifier + * notifier to be registered. + * @param jobClassName + * job class name on which to register. + */ + void registerJobClassNotification(MonitorListener notifier, + String jobClassName); + + /** + * Deregister the notifier for job class notification on some job class + * @param notifier + * notifier to be deregistered. + * @param jobClassName + * job class name on which to deregister. + */ + void deregisterJobClassNotification(MonitorListener notifier, + String jobClassName); + + /** + * Unregister this notifier from all its registered notifications. + * @param notifier + * notifier to be deregistered. + */ + void deregisterNotifications(MonitorListener notifier); +} diff --git a/azkaban/src/java/azkaban/monitor/MonitorInternalInterface.java b/azkaban/src/java/azkaban/monitor/MonitorInternalInterface.java new file mode 100644 index 0000000..282abe9 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/MonitorInternalInterface.java @@ -0,0 +1,127 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import azkaban.common.jobs.Job; +import azkaban.jobcontrol.impl.jobs.ResourceThrottledJob; +import azkaban.monitor.MonitorInterface.GlobalNotificationType; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.stats.ClassStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +/** + * Monitor interface used internally by Azkaban - primary use is to field events + * from the scheduler regarding the launching of workflows and jobs, and handing + * those events to the execution model, and to statistics aggregation. + */ +public interface MonitorInternalInterface { + + /** + * Enum regarding the types of workflow actions or events that are of + * interest capturing. + */ + public static enum WorkflowAction { + SCHEDULE_WORKFLOW, + START_WORKFLOW, + END_WORKFLOW, + UNSCHEDULE_WORKFLOW, + } + + /** + * Enum regarding the types of job actions or events that are of + * interest in capturing. + * + */ + public static enum JobAction { + START_WORKFLOW_JOB, + END_WORKFLOW_JOB, + } + + /** + * Captures a pure workflow event, as found in the scheduler typically. + * @param wfId + * String variant of int workflow id. + * @param time + * Time in epoch ms for this event. + * @param action + * Workflow action for this event. + * @param wfState + * workflow state (only used when action is EndJob, otherwise Nop expected.) + * @param rootJobName + * name of root workflow job (only used when action is ScheduleJob.) + */ + public void workflowEvent(String wfId, + long time, + WorkflowAction action, + WorkflowState wfState, + String rootJobName); + + /** + * Captures a job event as typically found in executable workflow class + * types. + * @param job + * job behind the job event. + * @param time - + * Time in epoch ms relevant to this event. + * @param jobAction + * job action for this event. + * @param jobState + * job state (only used when action is EndWorkflow, otherwise Nop expected.) + */ + public void jobEvent(Job job, + long time, + JobAction jobAction, + JobState jobState); + + /** + * Captures the wait time on getting resource locks. + * @param job + * resource throttled job. + * @param lockWaitTime + * time in ms of time logged waiting on locks. + */ + public void workflowResourceThrottledJobEvent(ResourceThrottledJob job, + long lockWaitTime); + + /** + * Method to execute the notification system that updates have occurred + * for the Global Stats, or that some workflow or job statistics object has + * been updated. + * @param eventType + * GlobalNotificationType of the notify action taken. + * @param statsObject + * the stats object that changed, global, workflow, or job depending on + * event type. + */ + public void executeGlobalNotify(GlobalNotificationType eventType, ClassStats statsObject); + + /** + * Method to execute the notification system that a specifc workflow class + * statistics object has been updated. + * @param wfStats + * NativeWorkflowClassStats that will be the argument to the notification. + */ + public void executeWorkflowClassNotify(NativeWorkflowClassStats wfStats); + + /** + * Method to execute the notification system that a specific job class + * statistics object has been updated. + * @param jobStats + * NativeJobClassStats that will be the argument to the notification. + */ + public void executeJobClassNotify(NativeJobClassStats jobStats); +} diff --git a/azkaban/src/java/azkaban/monitor/MonitorListener.java b/azkaban/src/java/azkaban/monitor/MonitorListener.java new file mode 100644 index 0000000..6ffd7f8 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/MonitorListener.java @@ -0,0 +1,50 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import azkaban.monitor.MonitorInterface.GlobalNotificationType; +import azkaban.monitor.stats.ClassStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +/** + * Notification interface for updates on global, workflow, and job statistics. + * + */ +public interface MonitorListener { + + /** + * Notification regarding global state, all workflows, all jobs. + * @param type + * the GlobalNotificationType representing the notification type./ + * @param statsObject + * the statistics object that changed, global, workflow, or job class. + */ + public void onGlobalNotify(GlobalNotificationType type, ClassStats statsObject); + + /** + * Notification that workflow class statistics has been updated. + * @param wfStats + * a snapshot copy of a workflow class stats. + */ + public void onWorkflowNotify(NativeWorkflowClassStats wfStats); + + /** + * Notification that job class statistics has been updated. + * @param jobStats + * a snapshot copy of a job class stats. + */ + public void onJobNotify(NativeJobClassStats jobStats); +} diff --git a/azkaban/src/java/azkaban/monitor/NotificationDetails.java b/azkaban/src/java/azkaban/monitor/NotificationDetails.java new file mode 100644 index 0000000..271586a --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/NotificationDetails.java @@ -0,0 +1,146 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import java.util.HashSet; +import java.util.Set; + +/** + * Class used to track the types of events a notifier has requested. + * These types fall into the following categories: + * 1) Global, all workflows, all jobs. + * 2) Specific workflow classes. + * 3) Specific job classes. + * + */ +public class NotificationDetails { + + /** + * The notifier whose notifications are being enumerated. + */ + private MonitorListener notifier; + + /** + * Set if interested in global statistics. + */ + private boolean globalStatsMonitor; + + /** + * Set if interested in notification on all workflow classes. + */ + private boolean allWorkflowClassStatsMonitor; + + /** + * Set if interested in notification on all job classes. + */ + private boolean allJobClassStatsMonitor; + + /** + * Set of all workflow classes of which notifier is interested. + */ + private Set globalWorkflowClasses = new HashSet(); + + /** + * Set of all job classes of which notifier is interested. + */ + private Set globalJobClasses = new HashSet(); + + /** + * Constructor. + * @param notifier + * the notifier whose details we will collect in this instance. + */ + public NotificationDetails(MonitorListener notifier) { + this.notifier = notifier; + } + + /** + * setter + * @param tf + * set if has global stats notification. + */ + public void setGlobalStatsMonitor(boolean tf) { + globalStatsMonitor = tf; + } + + /** + * getter + * @return true if has global stats notification. + */ + public boolean hasGlobalStatsMonitor() { + return globalStatsMonitor; + } + + /** + * setter + * @param tf + * set if has all workflow class changes notification. + */ + public void setAllWorkflowClassStatsMonitor(boolean tf) { + allWorkflowClassStatsMonitor = tf; + } + + /** + * getter + * @return true if has all workflow class stats change notification. + */ + public boolean hasAllWorkflowClassStatsMonitor() { + return allWorkflowClassStatsMonitor; + } + + /** + * setter + * @param tf + * set if has all job class changes notification. + */ + public void setAllJobClassStatsMonitor(boolean tf) { + allJobClassStatsMonitor = tf; + } + + /** + * getter + * @return true if has all job class stats change notification. + */ + public boolean hasAllJobClassStatsMonitor() { + return allJobClassStatsMonitor; + } + + /** + * getter + * @return set of names of all workflow root jobs notifier responds + * specifically to. + */ + public Set getGlobalWorkflowClasses() { + return globalWorkflowClasses; + } + + /** + * getter + * @return set of names of all job class names notifier responds + * to specifically. + */ + public Set getGlobalJobClasses() { + return globalJobClasses; + } + + /** + * getter + * @return the notifier which is th focus of this instance. + */ + public MonitorListener getNotifier() { + return notifier; + } + +} diff --git a/azkaban/src/java/azkaban/monitor/model/DelegatingJobModel.java b/azkaban/src/java/azkaban/monitor/model/DelegatingJobModel.java new file mode 100644 index 0000000..ef6c75d --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/DelegatingJobModel.java @@ -0,0 +1,56 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +/** + * Class that builds on JobModel for Azkaban delegating jobs. We are following along the line of + * Job inheritance in Azkaban - so the job models inherit from this class, e.g. RetryingJobModel. + * + */ +public class DelegatingJobModel extends JobExecutionModel { + + /** + * Identified the 'current' job model for the inner job. + */ + private JobExecutionModel innerJobModel; + + /** + * Constructor. + * @param jobName + * the name of this job + * @param workflow + * the workflow execution model to which this job belongs. + */ + public DelegatingJobModel(String jobName, WorkflowExecutionModel workflow) { + super(jobName, workflow); + } + + /** + * setter + * @param innerJobModel + * the corresponding inner job model for this job model. + */ + public void setInnerJobModel(JobExecutionModel innerJobModel) { + this.innerJobModel = innerJobModel; + } + + /** + * getter + * @return the inner job model for this delegating job model. + */ + public JobExecutionModel getInnerJobModel() { + return innerJobModel; + } +} diff --git a/azkaban/src/java/azkaban/monitor/model/ExecutionModel.java b/azkaban/src/java/azkaban/monitor/model/ExecutionModel.java new file mode 100644 index 0000000..e663af2 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/ExecutionModel.java @@ -0,0 +1,232 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +import java.util.List; +import java.util.Map; + +import azkaban.common.jobs.Job; +import azkaban.jobcontrol.impl.jobs.ResourceThrottledJob; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.stats.NativeGlobalStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +/** + * Interface to ExecutionModel which encapsulates managing the workflow + * and job model, as well as global statistics objects and notification + * management. + */ +public interface ExecutionModel { + + /** + * Tell the workflow model object that the workflow has been scheduled. + * @param rootJobName + * root job name of this workflow. + * @param time + * epoch time in ms for scheduling the workflow. + */ + public void scheduleWorkflow(String rootJobName, long time); + + /** + * Tell the workflow model object that the workflow has started. + * @param rootJobName + * root job name of this workflow. + * @param wfId + * the string version of int workflow id. + * @param time + * epoch time in ms for starting the workflow. + */ + public void startWorkflow(String rootJobName, String wfId, long time); + + /** + * Tell the workflow model object that the workflow has ended. + * @param wfId + * the string version of int workflow id. + * @param time + * epoch time in ms for ending the workflow. + * @param wfState + * the state of the workflow when it ended. + */ + public void endWorkflow(String wfId, long time, WorkflowState wfState); + + /** + * Tell the model object that a sub-job of it has started. + * @param time + * epoch time in ms for when this job started. + * @param job + * the Job that is being started. + */ + public void startWorkflowJob(long time, Job job); + + /** + * Tell the model object that a sub-job has ended. + * @param time + * epoch time in ms when this job ended. + * @param job + * the Job that is ended. + * @param jobState + * the state of the job when it ended. + */ + public void endWorkflowJob(long time, + Job job, + JobState jobState); + + /** + * Set a resource wait time for a job in a workflow. + * @param lockWaitTime + * time in ms logged for waiting on resources. + * @param job + * the ResourceThrottledJob + */ + public void resourceWaitTime(long lockWaitTime, ResourceThrottledJob job); + + /** + * Update global/class stats based on the workflow being scheduled. + * @param wfModel + * the workflow execution model being updated for scheduled. + */ + public void updateWorkflowScheduledStats(WorkflowExecutionModel wfModel); + + /** + * Update global/class stats based on the workflow being started. + * @param wfModel + * the workflow execution model being updated for started. + */ + public void updateWorkflowStartedStats(WorkflowExecutionModel wfModel); + + /** + * Update global/class stats based on the workflow being ended. + * @param wfModel + * the workflow execution model being updated for ending. + */ + public void updateWorkflowEndedStats(WorkflowExecutionModel wfModel); + + /** + * Update the global/class stats based on the workflow job being started. + * @param jobModel + * the job execution model being updated for started. + */ + public void updateJobStartedStats(JobExecutionModel jobModel); + + /** + * Update the global/class stats based on the workflow being completed. + * @param jobModel + * the job execution model being jpdated for ending. + */ + public void updateJobCompletedStats(JobExecutionModel jobModel); + + /** + * Update/notify based on wait time for resource throttled job. + * @param jobModel - ResourceThrottledJobModel + */ + public void updateResourceThrottledJobLockWaitTime(ResourceThrottledJobModel jobModel); + + /** + * Return a copy of the global statistics object. + * @return a copy of the global statistics. + */ + public NativeGlobalStats getGlobalStatsCopy(); + + /** + * Return a copy of all the known workflow class's being (or were) monitored. + * @return the list of root job names for the workflow class stats. + */ + public List getWorkflowClassIds(); + + /** + * Get a copy of the workflow class statistics for some id. + * @param wfRootJobName + * the root job name behind the workflow whose stats are requested. + * @return the NativeWorkflowClassStats for the root job name provided. + * + */ + public NativeWorkflowClassStats getWorkflowClassStatsById(String wfRootJobName); + + /** + * Get a copy of all the workflow class statistics. + * @return Map mapping root job name to + * the workflow class stats (copy) + */ + public Map getAllWorkflowClassStats(); + + /** + * Get a list of all the job class's being (or were) monitored. + * @return List a list list of job class names whose global stats are + * being collected. + */ + public List getJobClassIds(); + + /** + * Get a copy of a job class statistics object + * @param jobClassId + * class name whose statistics are being collected. + * @return the NativeJobClassStats corresponding to the job class name provided. + */ + public NativeJobClassStats getJobClassStatsById(String jobClassId); + + /** + * Return a copy of all the job class statistics. + * @return Map + */ + public Map getAllJobClassStats(); + + /** + * Return the number of workflows executions, active and completed. + * @return the number of workflow executions. + */ + public long getNumberOfWorkflows(); + + /** + * Return the number of completed workflow executions. + * @return the number completed workflow executions. + */ + public long getNumberOfCompletedWorkflows(); + + /** + * Return a list of all currently completed workflow models. + * @param reverseTime + * true indicates return list in reverse time order. + * @return List, a list of workflow execution models + * that have completed. + */ + public List getCompletedWorkflowModels(boolean reverseTime); + + /** + * Return a list of workflow models completed prior to a given time. + * @param endedBeforeTime + * Epoch time in ms prior to which workflows models should end. + * @param reverseTime + * true indicagtes return list in reverse time order. + * @return List, a list of workflow execution models + * that have completed. + */ + public List getCompletedWorkflowModels( + long endedBeforeTime, + boolean reverseTime); + + /** + * Clear out all the completed workflow models - used to free storage. + */ + public void clearCompletedWorkflows(); + + /** + * Clear out all workflow models completed before a given time from now. + * @param endedBeforeTime + * Epoch time in ms before which completed workflows are deleted. + */ + public void clearCompletedWorkflows(long endedBeforeTime); +} diff --git a/azkaban/src/java/azkaban/monitor/model/ExecutionModelImpl.java b/azkaban/src/java/azkaban/monitor/model/ExecutionModelImpl.java new file mode 100644 index 0000000..8d80158 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/ExecutionModelImpl.java @@ -0,0 +1,421 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.log4j.Logger; +import org.joda.time.DateTimeUtils; + +import azkaban.common.jobs.Job; +import azkaban.jobcontrol.impl.jobs.ResourceThrottledJob; +import azkaban.monitor.MonitorInternalInterface; +import azkaban.monitor.MonitorInterface.GlobalNotificationType; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.stats.GlobalStats; +import azkaban.monitor.stats.JobClassStats; +import azkaban.monitor.stats.NativeGlobalStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; +import azkaban.monitor.stats.WorkflowClassStats; + +/** + * Class that holds all workflow and job models, and also holds and manages + * the global statistics, including global stats, workflow class stats, and + * job class stats. + */ +public class ExecutionModelImpl implements ExecutionModel { + private static final Logger logger = Logger.getLogger(ExecutionModelImpl.class); + + /** + * Reference to internal notification methods. + */ + private MonitorInternalInterface monitor; + + /** + * Map model name to WorkflowModel. + */ + private Map wfModels = new ConcurrentHashMap(); + + /** + * Map from job name to WorkflowExecutionModel for all executions that are scheduled. + */ + private Map scheduledModels = new ConcurrentHashMap(); + + /** + * Global Azkaban Statistics - only one instance necessary. + */ + private GlobalStats globalStats; + + /** + * FlowDirectory to manage job mappings to flowid and delegate + */ + private FlowDirectory flowDirectory; + + /** + * Map workflow name to Workflow Class Statistics. + */ + private Map workflowClassStats = + new ConcurrentHashMap (); + + /** + * Map Job name to Job Class Statistics. + */ + private Map jobClassStats = + new ConcurrentHashMap(); + + /** + * Constructor. + * @param monitor + * MonitorInternalInterface + */ + public ExecutionModelImpl(MonitorInternalInterface monitor) { + this.monitor = monitor; + globalStats = new GlobalStats(); + + flowDirectory = FlowDirectory.getFlowDirectory(); + } + + //*************** All Basic Update Activity **************// + + public synchronized void scheduleWorkflow(String rootJobName, long time) { + // Assume only one scheduled workflow at a time. If one is already there, then + // we overwrite it. + WorkflowExecutionModel scheduledModel = new WorkflowExecutionModel(null, rootJobName); + scheduledModel.setScheduledTime(time); + scheduledModels.put(rootJobName, scheduledModel); + updateWorkflowScheduledStats(scheduledModel); + } + + public synchronized void startWorkflow(String rootJobName, String wfId, long time) { + // If the workflow model is scheduled, then let's remove it and promote it to the + // wfModels map. + WorkflowExecutionModel wfModel = scheduledModels.get(rootJobName); + if (wfModel == null) { + // no schedule exists + WorkflowExecutionModel existingWorkflow = wfModels.get(wfId); + if (wfModels.get(wfId) != null) { + // this is for retries. + wfModel = new WorkflowExecutionModel(wfId, rootJobName, existingWorkflow); + } else { + wfModel = new WorkflowExecutionModel(wfId, rootJobName); + } + } + else { + // job was scheduled, since it's starting remove it from scheduled. + scheduledModels.remove(rootJobName); + wfModel.setWorkflowId(wfId); + } + + wfModels.put(wfId, wfModel); + wfModel.setStartTime(time); + + updateWorkflowStartedStats(wfModel); + } + + public synchronized void endWorkflow(String wfId, long time, WorkflowState wfState) { + WorkflowExecutionModel wfModel = wfModels.get(wfId); + if (wfModel == null) { + logger.error("No workflow model exists with id: " + wfId); + return; + } + wfModel.setEndTime(time, wfState); + + updateWorkflowEndedStats(wfModel); + } + + public synchronized void startWorkflowJob(long time, Job job) { + String wfId = flowDirectory.getFlowId(job); + if (wfId == null) { + wfId = flowDirectory.mapJob(job); + if (wfId == null) { + logger.info("Could not find flow id for job: " + job.getClass().getSimpleName()); + return; + } + } + + WorkflowExecutionModel wfModel = wfModels.get(wfId); + if (wfModel == null) { + logger.error("No workflow model exists with id: " + wfId); + return; + } + JobExecutionModel jobModel = wfModel.startJob(job, time); + + updateJobStartedStats(jobModel); + } + + public synchronized void endWorkflowJob(long time, + Job job, + JobState jobState) { + String wfId = flowDirectory.getFlowId(job); + if (wfId == null) { + logger.info("Could not find workflow id for job: " + job.getClass().getSimpleName()); + return; + } + + WorkflowExecutionModel wfModel = wfModels.get(wfId); + if (wfModel == null) { + logger.error("No workflow model exists with id: " + wfId); + return; + } + JobExecutionModel jobExecutionModel = wfModel.endJob(job, time, jobState); + + updateJobCompletedStats(jobExecutionModel); + + flowDirectory.removeJobReference(job); + } + + public synchronized void resourceWaitTime(long lockWaitTime, ResourceThrottledJob job) { + String wfId = flowDirectory.getFlowId(job); + if (wfId == null) { + logger.info("Could not find workflow id for job: " + job.getClass().getSimpleName()); + return; + } + + WorkflowExecutionModel wfModel = wfModels.get(wfId); + if (wfModel == null) { + logger.error("No workflow model exists with id: " + wfId); + return; + } + ResourceThrottledJobModel jModel = wfModel.resourceWaitTime(job, lockWaitTime); + + updateResourceThrottledJobLockWaitTime(jModel); + } + + //*************** Model object stats update methods *************** + + public synchronized void updateWorkflowScheduledStats(WorkflowExecutionModel wfModel) { + // Update the global stats based on this action + globalStats.updateWorkflowScheduledStats(wfModel); + + // Update the workflow class stats based on this action + WorkflowClassStats wfClassStats = getWorkflowClassStats(wfModel.getRootJobName()); + wfClassStats.updateWorkflowScheduledStats(wfModel); + + sendWorkflowNotifications(wfModel); + } + + public synchronized void updateWorkflowStartedStats(WorkflowExecutionModel wfModel) { + // Update the global stats based on this action + globalStats.updateWorkflowStartedStats(wfModel); + + // Update the workflow class stats based on this action + WorkflowClassStats wfClassStats = getWorkflowClassStats(wfModel.getRootJobName()); + wfClassStats.updateWorkflowStartedStats(wfModel); + + sendWorkflowNotifications(wfModel); + } + + public synchronized void updateWorkflowEndedStats(WorkflowExecutionModel wfModel) { + globalStats.updateWorkflowEndedStats(wfModel); + + WorkflowClassStats wfClassStats = getWorkflowClassStats(wfModel.getRootJobName()); + wfClassStats.updateWorkflowEndedStats(wfModel); + + sendWorkflowNotifications(wfModel); + } + + private synchronized void sendWorkflowNotifications(WorkflowExecutionModel wfModel) { + monitor.executeGlobalNotify(GlobalNotificationType.GLOBAL_STATS_CHANGE, + new NativeGlobalStats(globalStats)); + + WorkflowClassStats wfClassStats = getWorkflowClassStats(wfModel.getRootJobName()); + monitor.executeWorkflowClassNotify(new NativeWorkflowClassStats( + wfClassStats)); + monitor.executeGlobalNotify( + GlobalNotificationType.ANY_WORKFLOW_CLASS_STATS_CHANGE, + new NativeWorkflowClassStats(wfClassStats)); + } + + public synchronized void updateJobStartedStats(JobExecutionModel jobModel) { + // Update the global stats based on this action. + globalStats.updateJobStartedStats(jobModel); + + // Update the correct job model stats. + JobClassStats jobClassStats = getJobClassStats(jobModel.getJobName()); + jobClassStats.updateJobStartedStats(jobModel); + + sendJobNotifications(jobModel); + } + + public synchronized void updateJobCompletedStats(JobExecutionModel jobModel) { + globalStats.updateJobCompletedStats(jobModel); + + JobClassStats jobClassStats = getJobClassStats(jobModel.getJobName()); + jobClassStats.updateJobCompletionStats(jobModel); + + sendJobNotifications(jobModel); + } + + private synchronized void sendJobNotifications(JobExecutionModel jobModel) { + monitor.executeGlobalNotify(GlobalNotificationType.GLOBAL_STATS_CHANGE, + new NativeGlobalStats(globalStats)); + + JobClassStats jobClassStats = getJobClassStats(jobModel.getJobName()); + monitor.executeJobClassNotify(new NativeJobClassStats(jobClassStats)); + monitor.executeGlobalNotify( + GlobalNotificationType.ANY_JOB_CLASS_STATS_CHANGE, + new NativeJobClassStats(jobClassStats)); + } + + public synchronized void updateResourceThrottledJobLockWaitTime(ResourceThrottledJobModel jobModel) { + JobClassStats jobClassStats = getJobClassStats(jobModel.getJobName()); + jobClassStats.updateResourceThrottledJobLockWaitTime(jobModel); + + monitor.executeJobClassNotify(new NativeJobClassStats(jobClassStats)); + } + + //*************** Accessors to statistical objects ***************// + + public synchronized NativeGlobalStats getGlobalStatsCopy() { + return new NativeGlobalStats(globalStats); + } + + public synchronized List getWorkflowClassIds() { + return new ArrayList(workflowClassStats.keySet()); + } + + public synchronized NativeWorkflowClassStats getWorkflowClassStatsById(String wfRootJobName) { + WorkflowClassStats wfClass = workflowClassStats.get(wfRootJobName); + if (wfClass == null) { + logger.info("No class stats found for workflow rootClass: " + wfRootJobName); + return null; + } + return new NativeWorkflowClassStats(workflowClassStats.get(wfRootJobName)); + } + + public synchronized Map getAllWorkflowClassStats() { + Map allStats = new HashMap(); + for (Entry entry : workflowClassStats.entrySet()) { + allStats.put(entry.getKey(), new NativeWorkflowClassStats(entry.getValue())); + } + return allStats; + } + + public synchronized List getJobClassIds() { + return new ArrayList(jobClassStats.keySet()); + } + + public synchronized NativeJobClassStats getJobClassStatsById(String jobClassId) { + JobClassStats jobClass = jobClassStats.get(jobClassId); + if (jobClass == null) { + logger.info("No job class stats exists for job: " + jobClassId); + return null; + } + return new NativeJobClassStats(jobClass); + } + + public synchronized Map getAllJobClassStats() { + Map allStats = new HashMap(); + for (Entry entry : jobClassStats.entrySet()) { + allStats.put(entry.getKey(), new NativeJobClassStats(entry.getValue())); + } + return allStats; + } + + private synchronized WorkflowClassStats getWorkflowClassStats(String rootJobName) { + WorkflowClassStats wfClass = workflowClassStats.get(rootJobName); + if (wfClass == null) { + wfClass = new WorkflowClassStats(rootJobName); + workflowClassStats.put(rootJobName, wfClass); + } + return wfClass; + } + + private synchronized JobClassStats getJobClassStats(String jobName) { + JobClassStats jobClass = jobClassStats.get(jobName); + if (jobClass == null) { + jobClass = new JobClassStats(jobName); + jobClassStats.put(jobName, jobClass); + } + return jobClass; + } + + public synchronized long getNumberOfWorkflows() { + return wfModels.size(); + } + + public synchronized long getNumberOfCompletedWorkflows() { + long counter = 0; + for (WorkflowExecutionModel wfModel : wfModels.values()) { + if (wfModel.isCompleted()) { + counter++; + } + } + return counter; + } + + public synchronized List getCompletedWorkflowModels(long endedBeforeTime, + boolean reverseTime) { + List workflowModels = new ArrayList(); + for (WorkflowExecutionModel wfModel : wfModels.values()) { + if (wfModel.isCompleted() && wfModel.getEndTime() <= endedBeforeTime) { + workflowModels.add(wfModel); + } + } + Collections.sort(workflowModels, new WFModelsComparator(reverseTime)); + return workflowModels; + } + + public synchronized List getCompletedWorkflowModels(boolean reverseTime) { + return getCompletedWorkflowModels(DateTimeUtils.currentTimeMillis(), reverseTime); + } + + public synchronized void clearCompletedWorkflows(long endedBeforeTime) { + List completedModels = getCompletedWorkflowModels(endedBeforeTime, false); + for (WorkflowExecutionModel wfModel : completedModels) { + wfModels.remove(wfModel.getWorkflowId()); + } + } + + public synchronized void clearCompletedWorkflows() { + clearCompletedWorkflows(DateTimeUtils.currentTimeMillis()); + } + + /** + * Comparator class used to sort a list of WorkflowExecutionModel's + * in reverse time order. + * + */ + private static class WFModelsComparator implements Comparator { + + private boolean reverse; + + /** + * Constructor + * @param reverse + * indicates (true) if reversing time order in comparison. + */ + public WFModelsComparator(boolean reverse) { + this.reverse = reverse; + } + + @Override + public int compare(WorkflowExecutionModel arg0, WorkflowExecutionModel arg1) { + int compare = (arg0.getStartTime() < arg1.getStartTime() ? -1 : + arg0.getStartTime() == arg1.getStartTime() ? 0 : 1); + return (compare == 0 ? 0 : reverse ? -compare : compare); + } + + } +} diff --git a/azkaban/src/java/azkaban/monitor/model/FlowDirectory.java b/azkaban/src/java/azkaban/monitor/model/FlowDirectory.java new file mode 100644 index 0000000..45c67a4 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/FlowDirectory.java @@ -0,0 +1,185 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.log4j.Logger; + +import azkaban.common.jobs.DelegatingJob; +import azkaban.common.jobs.Job; +import azkaban.common.utils.Props; +import azkaban.jobs.AbstractProcessJob; + +/** + * The point of this class is to track the mappings of job to flowId, and the + * mapping of job to its parent DelegatingJob when that occurs. + * This map is used by MonitorImpl and ExecutionModel. + * + */ +public final class FlowDirectory { + private static final Logger logger = Logger.getLogger(FlowDirectory.class); + + /** + * Maps job to the flow id of the workflow of which it is part. + */ + private Map jobToFlowMap = new HashMap(); + + /** + * Map a job to its parent delegating job parent. All instances should + * eventually back-map to a root most job which should have a flow id. + * Just to refresh, the job model only points forward, we need this back map + * at various occasions to get the delegating job parent directly from the inner job. + */ + private Map delegatingParentMap = new HashMap(); + + private volatile static FlowDirectory thisDirectory; + + private FlowDirectory() { } + + /** + * Accessor to the flow director singleton. + * @return FlowDirectory + */ + public static FlowDirectory getFlowDirectory() { + synchronized (FlowDirectory.class) { + if (thisDirectory == null) { + thisDirectory = new FlowDirectory(); + } + } + return thisDirectory; + } + + /** + * Used in testing to reset the singleton. + */ + public static void unsetFlowDirectory() { + synchronized (FlowDirectory.class) { + thisDirectory = null; + } + } + + /** + * Map the job to the flow id. The complexity here comes if job is a delegating job - + * we build the back map described earlier from inner job to parent delegating job. + * A main challenge here is to get the flowId - which we get here from the inner + * most job, by way of the 'azkaban.flow.id' property. + * Note: you should only find delegation depth at max 3. + * @param job + * Job to decompose and map into constituent jobs + * @return String version (int in string representation) of id of the workflow of which + * the job is a part. + */ + public synchronized String mapJob(Job job) { + Job jobCursor = job; + while (jobCursor != null) { + if (jobCursor instanceof DelegatingJob) { + DelegatingJob parentJob = (DelegatingJob)jobCursor; + delegatingParentMap.put(parentJob.getInnerJob(), parentJob); + jobCursor = parentJob.getInnerJob(); + } else { + // This is the innermost job - this is the gold piece we are looking for. + // We assume this is a process job which should have 'props' that identifies the + // flow id. We will use this for our map. + if (!(jobCursor instanceof AbstractProcessJob)) { + logger.warn("Non-Delegating job that is not a process job: " + + jobCursor.getId()); + return null; + } + AbstractProcessJob pJob = (AbstractProcessJob)jobCursor; + Props props = pJob.getProps(); + if (props == null) { + logger.warn("Inner job of delegating job sequence has no props: " + pJob.getId()); + return null; + } + String flowId = props.get("azkaban.flow.id"); + if (flowId == null) { + logger.warn("Inner job of delegating job sequence has no flow id: " + pJob.getId()); + return null; + } + + // Now that we determined the flowId, run through this sequence of jobs again, and + // map all to the flowId, so we can derive the flowId from a job later. + mapFlowIdForward(job, flowId); + return flowId; + } + } + return null; + } + + private void mapFlowIdForward(Job job, String flowId) { + Job jobCursor = job; + while (jobCursor != null) { + if (jobCursor instanceof DelegatingJob) { + DelegatingJob parentJob = (DelegatingJob)jobCursor; + jobToFlowMap.put(parentJob, flowId); + jobCursor = parentJob.getInnerJob(); + } else { + jobToFlowMap.put(jobCursor, flowId); + break; + } + } + } + + /** + * Get the delegating job whose inner job is given by the argument. + * @param job + * The job whose delegate parent is requested. + * @return DelegatingJob parent of the job + */ + public synchronized DelegatingJob getDelegatingJobParent(Job job) { + return delegatingParentMap.get(job); + } + + /** + * Return the workflow id for a given job. + * @param job + * The job of whose workflow id is requested. + * @return String version of the (int) workflow id of the job. + */ + public synchronized String getFlowId(Job job) { + return jobToFlowMap.get(job); + } + + /** + * Remove a job reference from the FlowDirectory [internal mappings]. + * @param job + * the job whose accounting is being removed from the flow directory. + */ + public synchronized void removeJobReference(Job job) { + DelegatingJob dJob = delegatingParentMap.get(job); + if (dJob != null) { + return; + } + + // If the job is a delegating job [that has finished presumably], we know + // the inner job is finished - so take the inner job out of both maps. + // However, + // if job has a delegating parent, the above will happen when its parent is + // completed, so skip. Otherwise, if job does not have a delegating parent, this is + // really the end of this job, and we remove its reference to its job flow, + // as no follow-on call will do that. + if (job instanceof DelegatingJob) { + delegatingParentMap.remove(((DelegatingJob)job).getInnerJob()); + jobToFlowMap.remove(((DelegatingJob)job).getInnerJob()); + if (delegatingParentMap.get(job) == null) { + jobToFlowMap.remove(job); + } + } else { + jobToFlowMap.remove(job); + } + } +} diff --git a/azkaban/src/java/azkaban/monitor/model/JobExecutionModel.java b/azkaban/src/java/azkaban/monitor/model/JobExecutionModel.java new file mode 100644 index 0000000..0b869e0 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/JobExecutionModel.java @@ -0,0 +1,184 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +import azkaban.app.LoggingJob; +import azkaban.common.jobs.Job; +import azkaban.jobcontrol.impl.jobs.ResourceThrottledJob; +import azkaban.jobcontrol.impl.jobs.RetryingJob; +import azkaban.monitor.MonitorInterface.JobState; + +/** + * Model class that encapsulates a running job instance, and variables there of + * that would help in global statistics. + * + */ +public class JobExecutionModel { + + /** + * Name of the job. + */ + protected String jobName; + + /** + * Ref. to workflow model of which this job is a part. + */ + protected WorkflowExecutionModel workflow; + + /** + * Start time for the job. + */ + protected long startTime; + + /** + * End time for the job. + */ + protected long endTime; + + /** + * Total execution (run) time for this job, to millisec. + */ + protected long executionTime; + + /** + * Final job state. + */ + protected JobState finalJobState; + + /** + * Indicator that job has completed. + */ + protected boolean completed; + + /** + * Constructor. + * @param jobName + * name of the job + * @param workflow + * workflow execution model to which job belongs + */ + public JobExecutionModel(String jobName, WorkflowExecutionModel workflow) { + this.workflow = workflow; + this.jobName = jobName; + } + + /** + * Set the start time for the job execution, and contact eModel to update stats. + * @param time + * epoch time in ms for the start for the job. + */ + public void setStartTime(long time) { + startTime = time; + } + + /** + * Set the end time for the job execution, and contact eModel to update stats. + * @param time + * epoch time in ms for the ending of the job. + * @param jobState + * state of the job when it ended. + */ + public void setEndTime(long time, JobState jobState) { + endTime = time; + executionTime = endTime - startTime; + finalJobState = jobState; + completed = true; + } + + /** + * getter + * @return the job name + */ + public String getJobName() { + return jobName; + } + + /** + * getter + * @return the execution time in ms. + */ + public long getExecutionTime() { + return executionTime; + } + + /** + * getter + * @return the job state of the job when it ended. + */ + public JobState getCompletionState() { + return finalJobState; + } + + /** + * getter + * @return the start time in epoch ms. + */ + public long getStartTime() { + return startTime; + } + + /** + * getter + * @return the end time in epoch ms. + */ + public long getEndTime() { + return endTime; + } + + /** + * getter + * @return the final job state. + */ + public JobState getFinalJobState() { + return finalJobState; + } + + /** + * getter + * @return true if completed. + */ + public boolean isCompleted() { + return completed; + } + + /** + * getter + * @return the workflow execution for this job execution. + */ + public WorkflowExecutionModel getWorkflowModel() { + return workflow; + } + + /** + * Public method to create the right kind of JobModel based on the given Job type. + * @param job + * the Job whose model is being created. + * @param workflow + * the workflow execution for which the job model is created. + * @return the JobExecutionModel + */ + public static JobExecutionModel createJobModel(Job job, WorkflowExecutionModel workflow) { + if (job instanceof RetryingJob) { + return new RetryingJobModel(job.getId(), workflow); + } else if (job instanceof ResourceThrottledJob) { + return new ResourceThrottledJobModel(job.getId(), workflow); + } else if (job instanceof LoggingJob) { + return new LoggingJobModel(job.getId(), workflow); + } else { + return new JobExecutionModel(job.getId(), workflow); + } + } + +} diff --git a/azkaban/src/java/azkaban/monitor/model/LoggingJobModel.java b/azkaban/src/java/azkaban/monitor/model/LoggingJobModel.java new file mode 100644 index 0000000..4bef85a --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/LoggingJobModel.java @@ -0,0 +1,33 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +/** + * Corresponding model for Logging Job. + * + */ +public class LoggingJobModel extends DelegatingJobModel { + + /** + * Constructor. + * @param jobName + * the name of the job which is being logged. + * @param workflow + * the workflow execution behind this job. + */ + public LoggingJobModel(String jobName, WorkflowExecutionModel workflow) { + super(jobName, workflow); + } +} diff --git a/azkaban/src/java/azkaban/monitor/model/ResourceThrottledJobModel.java b/azkaban/src/java/azkaban/monitor/model/ResourceThrottledJobModel.java new file mode 100644 index 0000000..cb3355b --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/ResourceThrottledJobModel.java @@ -0,0 +1,55 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +/** + * Job model for resource throttled jobs. + * + */ +public class ResourceThrottledJobModel extends DelegatingJobModel { + + /** + * Resource lock wait time in ms. + */ + private long lockWaitTime; + + /** + * Constructor. + * @param jobName + * the name of the job behind this job model. + * @param workflow + * the workflow execution to which this job belongs. + */ + public ResourceThrottledJobModel(String jobName, WorkflowExecutionModel workflow) { + super(jobName, workflow); + } + + /** + * Set the waiting time for the locks, and inform eModel to update stats. + * @param lockWaitTime + * the resource wait time in ms. + */ + public void setResourceLockWaitTime(long lockWaitTime) { + this.lockWaitTime = lockWaitTime; + } + + /** + * getter + * @return the lock wait time. + */ + public long getLockWaitTime() { + return lockWaitTime; + } +} diff --git a/azkaban/src/java/azkaban/monitor/model/RetryingJobModel.java b/azkaban/src/java/azkaban/monitor/model/RetryingJobModel.java new file mode 100644 index 0000000..6e3f316 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/RetryingJobModel.java @@ -0,0 +1,60 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +import java.util.LinkedList; + +/** + * Job model for retrying jobs. Each instance maintains a reverse list of executions, with + * the first member being the most recent. + * + */ +public class RetryingJobModel extends DelegatingJobModel { + + /** + * This is the list of executions, a stack, with the most current + * to the front. + */ + LinkedList executions = new LinkedList(); + + /** + * Constructor. + * @param jobName + * the job name behind this retrying job. + * @param workflow + * the workflow exection to which this job belongs. + */ + public RetryingJobModel(String jobName, WorkflowExecutionModel workflow) { + super(jobName, workflow); + } + + /** + * Note that in this case the set inner job model actually appends the model + * to the stack of executions. + */ + public synchronized void setInnerJobModel(JobExecutionModel jobModel) { + super.setInnerJobModel(jobModel); + // This is one of our new executions + executions.addFirst(jobModel); + } + + /** + * getter + * @return the number of tries made on the inner job. + */ + public synchronized long getNumberOfTries() { + return executions.size(); + } +} diff --git a/azkaban/src/java/azkaban/monitor/model/WorkflowExecutionModel.java b/azkaban/src/java/azkaban/monitor/model/WorkflowExecutionModel.java new file mode 100644 index 0000000..325aab8 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/model/WorkflowExecutionModel.java @@ -0,0 +1,401 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.model; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.log4j.Logger; + +import azkaban.common.jobs.DelegatingJob; +import azkaban.common.jobs.Job; +import azkaban.jobcontrol.impl.jobs.ResourceThrottledJob; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInterface.WorkflowState; + +/** + * Model class that encapsulates a running workflow, and variables thereof that + * would assist in determining global statistics. + * + */ +public class WorkflowExecutionModel { + private static final Logger logger = Logger.getLogger(WorkflowExecutionModel.class); + + /** + * If the workflow is execution multiple time - with same id - + * this points to the last older version. + */ + private WorkflowExecutionModel olderExecution; + + /** + * The workflow id - string version of integer id value. + */ + private String wfId; + + /** + * Name of root job to workflow. + */ + private String rootJobName; + + /** + * Time workflow was scheduled. + */ + private long scheduledTime; + + /** + * Start time of workflow. + */ + private long startTime; + + /** + * End time of workflow. + */ + private long endTime; + + /** + * Amount of time spent pending (in millisec.). + */ + private long pendingTime; + + /** + * Amount of time spent in execution (in millisec.). + */ + private long executionTime; + + /** + * Total elapsed time (scheduled to termination) in millisec. + */ + private long elapsedTime; + + /** + * Final state of workflow. + */ + private WorkflowState finalWorkflowState = WorkflowState.UNKNOWN; + + /** + * Indication of workflow completion. + */ + private boolean completed; + + /** + * Flow directory to managing job delegating map. + */ + private FlowDirectory flowDirectory; + + /** + * Map a Job to the job model used in this workflow execution. + */ + private Map jobModelMapper = new ConcurrentHashMap(); + + /** + * List of job executions belonging to this workflow. + */ + private List wfExecutions = new ArrayList(); + + /** + * Constructor. + * @param wfId + * string variant of int workflow id assigned to workflow by engine. + * @param rootJobName + * name of the root job behind this workflow. + */ + public WorkflowExecutionModel(String wfId, String rootJobName) { + this.wfId = wfId; + this.rootJobName = rootJobName; + + flowDirectory = FlowDirectory.getFlowDirectory(); + } + + /** + * Constructor, accounting for re-execution with same workflow id. + * @param wfId + * string variant of int workflow id assigned to workflow by engine. + * @param rootJobName + name of the root job behind this workflow. + * @param olderExecution + * the prior execution this execution supersedes e.g. if workflow + * is restarted with same wfId. + */ + public WorkflowExecutionModel(String wfId, + String rootJobName, + WorkflowExecutionModel olderExecution) { + this(wfId, rootJobName); + this.olderExecution = olderExecution; + } + + /** + * Set the scheduled time. + * @param time + * epoch time in ms workflow was scheduled. + */ + public void setScheduledTime(long time) { + scheduledTime = time; + } + + /** + * Set the start time. + * @param time + * epoch time in ms workflow was started. + */ + public void setStartTime(long time) { + startTime = time; + + if (scheduledTime == 0) { + setScheduledTime(startTime); // no explicit schedule. + } + + pendingTime = startTime - scheduledTime; + } + + /** + * Set the end time for the full workflow. + * @param time + * epoch time this workflow was ended. + * @param wfState + * state of workflow execution when ended. + */ + public void setEndTime(long time, WorkflowState wfState) { + endTime = time; + + if (startTime == 0) { + String errorMsg = "Workflow start time never set for wf id:" + wfId; + logger.error(errorMsg); + return; + } + + executionTime = endTime - startTime; + + if (scheduledTime == 0) { + String errorMsg = "Workflow scheduled time never set for wf id:" + wfId; + logger.error(errorMsg); + return; + } + + elapsedTime = endTime - scheduledTime; + + finalWorkflowState = wfState; + + // Flag completed after final update of stats. + completed = true; + } + + /** + * Indicate that a job started in this workflow. + * @param job + * Job that was started in this workflow. + * @param time + * epoch time in ms job was started + * @return JobExecutionModel for started job. + */ + public JobExecutionModel startJob(Job job, long time) { + JobExecutionModel jobModel = jobModelMapper.get(job); + if (jobModel == null) { + jobModel = createJobModel(job); + } + jobModel.setStartTime(time); + return jobModel; + } + + /** + * Indicate that a job ended in this workflow. + * @param job + * Job has has ended. + * @param time + * epoch based time in ms when job ended. + * @param jobState + * state of job when it ended. + * @return JobExecutionModel for job. + */ + public JobExecutionModel endJob(Job job, long time, JobState jobState) { + JobExecutionModel jobModel = jobModelMapper.get(job); + if (jobModel == null) { + logger.debug("null job execution model"); + return null; + } + jobModel.setEndTime(time, jobState); + jobModelMapper.remove(job); // We are done with this job model. + + flowDirectory.removeJobReference(job); + return jobModel; + } + + /** + * Set the the resource wait time on the resource throttled job. + * @param job + * ResourceThrottledJob whose lock time is being set. + * @param lockWaitTime + * time in ms for resource lock + * @return ResourceThrottledJobModel for this job. + */ + public ResourceThrottledJobModel resourceWaitTime(ResourceThrottledJob job, long lockWaitTime) { + JobExecutionModel jobModel = jobModelMapper.get(job); + if (jobModel == null) { + logger.debug("null job execution model"); + return null; + } + if (!(jobModel instanceof ResourceThrottledJobModel)) { + logger.debug("job execution model not ResourceThrottledModel"); + return null; + } + ((ResourceThrottledJobModel)jobModel).setResourceLockWaitTime(lockWaitTime); + return (ResourceThrottledJobModel)jobModel; + } + + /** + * Create a job model based on the given job, with due consideration if it is + * a delegating job - which needs the corresponding DelegatingJobModel. + * @param job + * Job whose job execution model is being created. + * @return JobExecutionModel for job. + */ + private JobExecutionModel createJobModel(Job job) { + JobExecutionModel jobModel = JobExecutionModel.createJobModel(job, this); + + DelegatingJob delegatingJob = flowDirectory.getDelegatingJobParent(job); + if (delegatingJob != null) { + // This means that the job is the inner job of a delegating job; + DelegatingJobModel delegatingJobModel = (DelegatingJobModel)jobModelMapper.get(delegatingJob); + // The following provides a kind of notification to the delegating parent to account + // for its inner job accounting. + delegatingJobModel.setInnerJobModel(jobModel); + } + jobModelMapper.put(job, jobModel); + wfExecutions.add(jobModel); + return jobModel; + } + + /** + * Get all the workflow's execution jobs by name. + * @return a list of string names. + */ + public List getWorkflowJobNames() { + List jobnames = new ArrayList(); + for (JobExecutionModel jobModel : wfExecutions) { + jobnames.add(jobModel.getJobName()); + } + return jobnames; + } + + /** + * Get all workflow's job executions. + * @return a list of JobExecutionModels + */ + public List getWorkflowJobExecutions() { + List jobModels = new ArrayList(); + for (JobExecutionModel jobModel : wfExecutions) { + jobModels.add(jobModel); + } + return jobModels; + } + + /** + * getter + * @return workflow pending time in ms. + */ + public long getPendingTime() { + return pendingTime; + } + + /** + * getter + * @return execution time in ms. + */ + public long getExecutionTime() { + return executionTime; + } + + /** + * getter + * @return pending + execution time in ms. + */ + public long getElapsedTime() { + return elapsedTime; + } + + /** + * getter + * @return final WorkflowState value. + */ + public WorkflowState getCompletionState() { + return finalWorkflowState; + } + + /** + * getter + * @return the workflow id as string variant of int workflow id. + */ + public String getWorkflowId() { + return wfId; + } + + /** + * getter + * @return the root job name of workflow. + */ + public String getRootJobName() { + return rootJobName; + } + + /** + * getter + * @return true if workflow is completed. + */ + public boolean isCompleted() { + return completed; + } + + /** + * getter + * @return time in epoch ms for start time. + */ + public long getStartTime() { + return startTime; + } + + /** + * getter + * @return time in epcoh ms for end time. + */ + public long getEndTime() { + return endTime; + } + + /** + * getter + * @return return the final workflow state. + */ + public WorkflowState getFinalWorkflowState() { + return finalWorkflowState; + } + + /** + * getter + * @return get the next older execution model. + */ + public WorkflowExecutionModel getOlderExecutionModel() { + return olderExecution; + } + + /** + * sets the wfId + * @param wfId to set + */ + public void setWorkflowId(String wfId) { + this.wfId = wfId; + } + +} diff --git a/azkaban/src/java/azkaban/monitor/stats/ClassStats.java b/azkaban/src/java/azkaban/monitor/stats/ClassStats.java new file mode 100644 index 0000000..df41f86 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/stats/ClassStats.java @@ -0,0 +1,29 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.stats; + +/** + * General statistics class, providing an object foundation (superclass) for all + * statistics classes. + */ +public class ClassStats { + + /** + * Constructor. + */ + public ClassStats() { + } + +} diff --git a/azkaban/src/java/azkaban/monitor/stats/GlobalStats.java b/azkaban/src/java/azkaban/monitor/stats/GlobalStats.java new file mode 100644 index 0000000..5125837 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/stats/GlobalStats.java @@ -0,0 +1,125 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.stats; + +import org.apache.log4j.Logger; + +import azkaban.monitor.model.DelegatingJobModel; +import azkaban.monitor.model.JobExecutionModel; +import azkaban.monitor.model.WorkflowExecutionModel; + +/** + * Global statistics about what is running in Azkaban generally, e.g. number of jobs. + */ +public class GlobalStats extends NativeGlobalStats { + private static final Logger logger = Logger.getLogger(GlobalStats.class); + + /** + * Constructor. + */ + public GlobalStats() { + super(); + startTime = System.currentTimeMillis(); + } + + /** + * Update global statistics based on the scheduling of a workflow. + * @param wfModel + * the workflow execution that was scheduled. + */ + public synchronized void updateWorkflowScheduledStats(WorkflowExecutionModel wfModel) { + totalWorkflowsScheduled++; + totalWorkflowsPending = totalWorkflowsScheduled - totalWorkflowsStarted; + } + + /** + * Update global statistics based on the starting of a workflow. + * @param wfModel + * the workflow model that started. + */ + public synchronized void updateWorkflowStartedStats(WorkflowExecutionModel wfModel) { + totalWorkflowsStarted++; + totalWorkflowsPending = totalWorkflowsScheduled - totalWorkflowsStarted; + highFlowId = Math.max(highFlowId, Long.parseLong(wfModel.getWorkflowId())); + } + + @Override + public final Object clone() throws CloneNotSupportedException { + throw new CloneNotSupportedException(); + } + + /** + * Update global statistics based on workflow end. + * @param wfModel + * the workflow execution which ended. + */ + public synchronized void updateWorkflowEndedStats(WorkflowExecutionModel wfModel) { + switch (wfModel.getCompletionState()) { + case SUCCESSFUL: + totalWorkflowsSuccessful++; + break; + case FAILED: + totalWorkflowsFailed++; + break; + case CANCELED: + totalWorkflowsCanceled++; + break; + default: + logger.debug("Unknown WorkflowState."); + break; + } + } + + /** + * Update global statistics based on job started. + * @param jobModel + * the job executiobn that started. + */ + public synchronized void updateJobStartedStats(JobExecutionModel jobModel) { + // Don't count delegated activities, as they mis-lead. You can wind up + // double or triple counting actual starts - only the inner-most jobs. + if (jobModel instanceof DelegatingJobModel) { + return; + } + totalJobsStarted++; + } + + /** + * Update global statistics based on job completed. + * @param jobModel + * the job execution that completed. + */ + public synchronized void updateJobCompletedStats(JobExecutionModel jobModel) { + // Don't count delegated activities, as they mis-lead. You can wind up + // double or triple counting actual starts - only the inner-most jobs. + if (jobModel instanceof DelegatingJobModel) { + return; + } + switch (jobModel.getCompletionState()) { + case SUCCESSFUL: + totalJobsSuccessful++; + break; + case FAILED: + totalJobsFailed++; + break; + case CANCELED: + totalJobsCanceled++; + break; + default: + logger.debug("Unknown JobState."); + break; + } + } +} diff --git a/azkaban/src/java/azkaban/monitor/stats/JobClassStats.java b/azkaban/src/java/azkaban/monitor/stats/JobClassStats.java new file mode 100644 index 0000000..72fa5b3 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/stats/JobClassStats.java @@ -0,0 +1,214 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.stats; + +import org.apache.commons.math.stat.descriptive.SynchronizedSummaryStatistics; +import org.apache.log4j.Logger; + +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.model.JobExecutionModel; +import azkaban.monitor.model.LoggingJobModel; +import azkaban.monitor.model.ResourceThrottledJobModel; +import azkaban.monitor.model.RetryingJobModel; + +/** + * Class that collect global statistics over the runs of a particular job class. + * This class includes statistics associated with the job class if it incorporates retries, + * resource throttling, and logging. + * Statistics are of the nature of run counts, runtime averages, and many other counts + * and statistics. + * + */ +public class JobClassStats extends NativeJobClassStats { + private static final Logger logger = Logger.getLogger(JobClassStats.class); + + /** + * Stats mechanism for all non-delegating successful jobs. + */ + private SynchronizedSummaryStatistics normalJobStats = new SynchronizedSummaryStatistics(); + + /** + * Stats mechanism for all non-delegating failed jobs. + */ + private SynchronizedSummaryStatistics failedJobStats = new SynchronizedSummaryStatistics(); + + /** + * Stats mechanism for all non-delegating canceled jobs. + */ + private SynchronizedSummaryStatistics canceledJobStats = new SynchronizedSummaryStatistics(); + + /** + * Stats mechanism for all retry jobs. + */ + private SynchronizedSummaryStatistics retryJobStats = new SynchronizedSummaryStatistics(); + + /** + * Stats mechanism for all job retries. + */ + private SynchronizedSummaryStatistics retryJobRetriesStats = new SynchronizedSummaryStatistics(); + + /** + * Stats mechanism for all resource throttled jobs. + */ + private SynchronizedSummaryStatistics resourceThrottledJobStats = new SynchronizedSummaryStatistics(); + + /** + * Primary constructor for JobClassStats. + * @param jobClassName + * the job class name for these statistics. + */ + public JobClassStats(String jobClassName) { + super(); + this.jobClassName = jobClassName; + } + + /** + * Update job statistics based on a job starting. + * @param jobModel + * the job execution model used to update started stats. + */ + public synchronized void updateJobStartedStats(JobExecutionModel jobModel) { + if (jobModel instanceof RetryingJobModel) { + retryJob = true; + numRetryJobStarts++; + } else if (jobModel instanceof ResourceThrottledJobModel) { + resourceThrottledJob = true; + numResourceThrottledStarted++; + } else if (jobModel instanceof LoggingJobModel) { + loggingJob = true; + numLoggingJobStarts++; + } else { + numTimesJobStarted++; + } + if (lastTimeJobStarted == 0 || + lastTimeJobStarted < jobModel.getStartTime()) { + lastTimeJobStarted = jobModel.getStartTime(); + } + } + + /** + * Update job statistics based on job ending. + * @param jobModel + * the job execution model used to update completion stats. + */ + public synchronized void updateJobCompletionStats(JobExecutionModel jobModel) { + if (jobModel instanceof RetryingJobModel) { + updateRetryJobCompletionStats((RetryingJobModel)jobModel); + } else if (!(jobModel instanceof ResourceThrottledJobModel) && + !(jobModel instanceof LoggingJobModel)) { + updateNormalJobCompletionStats(jobModel); + } + } + + /** + * Update job statistics based on a retry job. + * @param jobModel + * the retrying model used to update retry completion stats. + */ + private synchronized void updateRetryJobCompletionStats(RetryingJobModel jobModel) { + switch (jobModel.getFinalJobState()) { + case SUCCESSFUL: + numRetryJobSuccessful++; + break; + case FAILED: + numRetryJobFailures++; + break; + case CANCELED: + numRetryJobCanceled++; + break; + default: + logger.debug("Unknown JobState."); + return; + } + + /** + * Only take average and std dev for successful jobs. + */ + if (jobModel.getFinalJobState() == JobState.SUCCESSFUL) { + + long incTime = jobModel.getEndTime() - + jobModel.getStartTime(); + retryJobStats.addValue(incTime); + avgRetryJobRunTime = retryJobStats.getMean(); + stdRetryJobRunTime = retryJobStats.getStandardDeviation(); + } + + retryJobRetriesStats.addValue(jobModel.getNumberOfTries()); + numJobTries = (long)retryJobRetriesStats.getSum(); + avgNumJobTries = retryJobRetriesStats.getMax(); + } + + /** + * Update a non delegating job's statistics. + * @param jobModel + * the job execution model used to update job completion stats. + */ + private void updateNormalJobCompletionStats(JobExecutionModel jobModel) { + SynchronizedSummaryStatistics jobStats = null; + switch (jobModel.getFinalJobState()) { + case SUCCESSFUL: + numTimesJobSuccessful++; + jobStats = normalJobStats; + break; + case FAILED: + numTimesJobFailed++; + jobStats = failedJobStats; + break; + case CANCELED: + numTimesJobCanceled++; + jobStats = canceledJobStats; + break; + default: + logger.debug("Unknown JobState."); + return; + } + + /** + * Take average and std dev for the different job states. + */ + long incTime = jobModel.getEndTime() - + jobModel.getStartTime(); + jobStats.addValue(incTime); + double avg = jobStats.getMean(); + double std = jobStats.getStandardDeviation(); + switch(jobModel.getFinalJobState()) { + case SUCCESSFUL: + avgJobRunTime = avg; + stdJobRunTime = std; + break; + case FAILED: + avgJobFailedTime = avg; + stdJobFailedTime = std; + break; + case CANCELED: + avgJobCanceledTime = avg; + stdJobCanceledTime = std; + break; + } + } + + /** + * Update the resource throttled job wait time on locks. + * @param jobModel + * the resource throttled job model used to update resource wait times. + */ + public void updateResourceThrottledJobLockWaitTime(ResourceThrottledJobModel jobModel) { + resourceThrottledJobStats.addValue(jobModel.getLockWaitTime()); + totalResourceThrottledWaitTime = resourceThrottledJobStats.getSum(); + avgResourceThrottledWaitTime = resourceThrottledJobStats.getMean(); + stdResourceThrottledWaitTime = resourceThrottledJobStats.getStandardDeviation(); + } + +} diff --git a/azkaban/src/java/azkaban/monitor/stats/NativeGlobalStats.java b/azkaban/src/java/azkaban/monitor/stats/NativeGlobalStats.java new file mode 100644 index 0000000..82dc0a7 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/stats/NativeGlobalStats.java @@ -0,0 +1,226 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.stats; + + +/** + * Base data-version of GlobalStats comprising all data fields and getters. + * Includes constructor for building clone of current values. Used to acquire + * stable version of actual NativeGlobalStats, for user. + * This class is not thread safe. + * + */ +public class NativeGlobalStats extends ClassStats { + /** + * Number of workflows scheduled. + */ + protected long totalWorkflowsScheduled; + + /** + * Number of workflows started. + */ + protected long totalWorkflowsStarted; + + /** + * Number of workflows that completed successfully. + */ + protected long totalWorkflowsSuccessful; + + /** + * Number of workflows that failed. + */ + protected long totalWorkflowsFailed; + + /** + * Number of workflows that were canceled. + */ + protected long totalWorkflowsCanceled; + + /** + * Number of currently pending workflows. + */ + protected long totalWorkflowsPending; + + /** + * Number of jobs started. + */ + protected long totalJobsStarted; + + /** + * Number of jobs that terminated successfully. + */ + protected long totalJobsSuccessful; + + /** + * Number of jobs that failed. + */ + protected long totalJobsFailed; + + /** + * Number of jobs that were canceled. + */ + protected long totalJobsCanceled; + + /** + * Highest workflow id started. + */ + protected long highFlowId; + + /** + * Time of most recent workflow started (to millisec.) + */ + protected long startTime; + + /** + * How long total monitoring has been occurring (to millisec.) + */ + protected long timeMonitoring; + + /** + * Primary constructor for the global statistics object. + */ + public NativeGlobalStats() { + super(); + } + + /** + * Copy constructor to construct duplicate based on what is in the master copy. + * Includes only the critical statistics. + * @param ngs + * the global stats being copied. + */ + public NativeGlobalStats(NativeGlobalStats ngs) { + super(); // Do not pass monitor to copies. + synchronized (ngs) { + this.totalWorkflowsScheduled = ngs.totalWorkflowsScheduled; + this.totalWorkflowsStarted = ngs.totalWorkflowsStarted; + this.totalWorkflowsSuccessful = ngs.totalWorkflowsSuccessful; + this.totalWorkflowsFailed = ngs.totalWorkflowsFailed; + this.totalWorkflowsCanceled = ngs.totalWorkflowsCanceled; + this.totalWorkflowsPending = ngs.totalWorkflowsPending; + this.totalJobsStarted = ngs.totalJobsStarted; + this.totalJobsSuccessful = ngs.totalJobsSuccessful; + this.totalJobsFailed = ngs.totalJobsFailed; + this.totalJobsCanceled = ngs.totalJobsCanceled; + this.highFlowId = ngs.highFlowId; + this.startTime = ngs.startTime; + this.timeMonitoring = System.currentTimeMillis() - ngs.startTime; + } + } + + /** + * getter + * @return the number of workflows scheduled. + */ + public final long getTotalWorkflowsScheduled() { + return totalWorkflowsScheduled; + } + + /** + * getter + * @return the number of workflows started. + */ + public final long getTotalWorkflowsStarted() { + return totalWorkflowsStarted; + } + + /** + * getter + * @return the number of workflows successful. + */ + public final long getTotalWorkflowsSuccessful() { + return totalWorkflowsSuccessful; + } + + /** + * getter + * @return the number of workflows failed. + */ + public final long getTotalWorkflowsFailed() { + return totalWorkflowsFailed; + } + + /** + * getter + * @return the number of workflows canceled. + */ + public final long getTotalWorkflowsCanceled() { + return totalWorkflowsCanceled; + } + + /** + * getter + * @return the number of workflows pending. + */ + public final long getTotalWorkflowsPending() { + return totalWorkflowsPending; + } + + /** + * getter + * @return the number of jobs started. + */ + public final long getTotalJobsStarted() { + return totalJobsStarted; + } + + /** + * getter + * @return the number of successful jobs. + */ + public final long getTotalJobsSuccessful() { + return totalJobsSuccessful; + } + + /** + * getter + * @return the number of failed jobs. + */ + public final long getTotalJobsFailed() { + return totalJobsFailed; + } + + /** + * getter + * @return the number of jobs canceled. + */ + public final long getTotalJobsCanceled() { + return totalJobsCanceled; + } + + /** + * getter + * @return the highest flow id recorded. + */ + public final long getHighFlowId() { + return highFlowId; + } + + /** + * getter + * @return the amount of time in ms monitoring. + */ + public final long getTimeMonitoring() { + return timeMonitoring; + } + + /** + * getter + * @return the epoch time in ms when monitoring started. + */ + public final long getStartTime() { + return startTime; + } +} diff --git a/azkaban/src/java/azkaban/monitor/stats/NativeJobClassStats.java b/azkaban/src/java/azkaban/monitor/stats/NativeJobClassStats.java new file mode 100644 index 0000000..e7a2ac6 --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/stats/NativeJobClassStats.java @@ -0,0 +1,449 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.stats; + +/** + * Base data-version of JobClassStats comprising all data fields and getters. + * Includes constructor for building clone of current values. Used to acquire + * stable version of actual NativeJobClassStats, for user. + * This class is not thread safe. + */ +public class NativeJobClassStats extends ClassStats { + /** + * Name of the job class. + */ + protected String jobClassName; + + /** + * Number of times this job was started. + */ + protected long numTimesJobStarted; + + /** + * Number of times this job ended successfully. + */ + protected long numTimesJobSuccessful; + + /** + * Number of times this job ended in failure. + */ + protected long numTimesJobFailed; + + /** + * Number of times this job was canceled. + */ + protected long numTimesJobCanceled; + + /** + * Average runtime for this job, as a successful run. + */ + protected double avgJobRunTime; + + /** + * Standard deviation of job runtime, as a successful run. + */ + protected double stdJobRunTime; + + /** + * Average runtime for this job, as failed job. + */ + protected double avgJobFailedTime; + + /** + * Standard deviation of job, as a failed job. + */ + protected double stdJobFailedTime; + + /** + * Average runtime for this job, as a canceled job. + */ + protected double avgJobCanceledTime; + + /** + * Standard deviation of job, as a canceled job. + */ + protected double stdJobCanceledTime; + + /** + * Most recent start time for this job class. + */ + protected long lastTimeJobStarted; + + /** + * Indicates if this is a retry job. + */ + protected boolean retryJob; + + /** + * Indicates if this is a resource throttled job. + */ + protected boolean resourceThrottledJob; + + /** + * Indicates if this is a logging job. + */ + protected boolean loggingJob; + + /** + * Number of times the retry job (that is, the job that does retries) starts. + */ + protected long numRetryJobStarts; + + /** + * Number of times the retry job itself was successful. + */ + protected long numRetryJobSuccessful; + + /** + * Number of times the retry job itself failed. + */ + protected long numRetryJobFailures; + + /** + * Number of times the retry job itself was canceled. + */ + protected long numRetryJobCanceled; + + /** + * The total number of job tries - should be equal to num started. + */ + protected long numJobTries; + + /** + * Average number of retries per retry job run. + */ + protected double avgNumJobTries; + + /** + * Average run time for the retry job itself [includes its retries.] + */ + protected double avgRetryJobRunTime; + + /** + * Standard deviation of the retry job runtime itself. + */ + protected double stdRetryJobRunTime; + + /** + * If this is a resource throttled jobs, this the the total lock waiting time (in ms). + */ + protected double totalResourceThrottledWaitTime; + + /** + * This is the average resource throttled waiting time (in ms). + */ + protected double avgResourceThrottledWaitTime; + + /** + * Standard deviation resource throttled waiting time (in ms). + */ + protected double stdResourceThrottledWaitTime; + + /** + * This is the total number of resouce throttled starts. + */ + protected long numResourceThrottledStarted; + + /** + * Number of stats as a logging job. + */ + protected long numLoggingJobStarts; + + + /** + * Primary constructor for JobClassStats. + */ + public NativeJobClassStats() { + super(); + } + + /** + * Copy constructor to construct duplicate based on what is in the master copy. + * Includes only the critical statistics. + * @param njcs + * NativeJobClassStats being copied. + */ + public NativeJobClassStats(NativeJobClassStats njcs) { + super(); // Do not pass monitor to copies. + synchronized (njcs) { + this.jobClassName = njcs.jobClassName; + this.numTimesJobStarted = njcs.numTimesJobStarted; + this.numTimesJobSuccessful = njcs.numTimesJobSuccessful; + this.numTimesJobFailed = njcs.numTimesJobFailed; + this.numTimesJobCanceled = njcs.numTimesJobCanceled; + this.avgJobRunTime = njcs.avgJobRunTime; + this.stdJobRunTime = njcs.stdJobRunTime; + this.avgJobFailedTime = njcs.avgJobFailedTime; + this.stdJobFailedTime = njcs.stdJobFailedTime; + this.avgJobCanceledTime = njcs.avgJobCanceledTime; + this.stdJobCanceledTime = njcs.stdJobCanceledTime; + this.lastTimeJobStarted = njcs.lastTimeJobStarted; + this.retryJob = njcs.retryJob; + this.resourceThrottledJob = njcs.resourceThrottledJob; + this.loggingJob = njcs.loggingJob; + + this.numRetryJobStarts = njcs.numRetryJobStarts; + this.numRetryJobSuccessful = njcs.numRetryJobSuccessful; + this.numRetryJobFailures = njcs.numRetryJobFailures; + this.numRetryJobCanceled = njcs.numRetryJobCanceled; + this.numJobTries = njcs.numJobTries; + this.avgNumJobTries = njcs.avgNumJobTries; + this.avgRetryJobRunTime = njcs.avgRetryJobRunTime; + this.stdRetryJobRunTime = njcs.stdRetryJobRunTime; + this.totalResourceThrottledWaitTime = njcs.totalResourceThrottledWaitTime; + this.avgResourceThrottledWaitTime = njcs.avgResourceThrottledWaitTime; + this.stdResourceThrottledWaitTime = njcs.stdResourceThrottledWaitTime; + this.numResourceThrottledStarted = njcs.numResourceThrottledStarted; + this.numLoggingJobStarts = njcs.numLoggingJobStarts; + } + } + + @Override + public final Object clone() throws CloneNotSupportedException { + throw new CloneNotSupportedException(); + } + + /** + * getter + * @return job class name + */ + public final String getJobClassName() { + return jobClassName; + } + + /** + * getter + * @return number of times job class started. + */ + public final long getNumTimesJobStarted() { + return numTimesJobStarted; + } + + /** + * getter + * @return number of times job class failed. + */ + public final long getNumTimesJobFailed() { + return numTimesJobFailed; + } + + /** + * getter + * @return number of times job class successful. + */ + public final long getNumTimesJobSuccessful() { + return numTimesJobSuccessful; + } + + /** + * getter + * @return number of times job class canceled. + */ + public final long getNumTimesJobCanceled() { + return numTimesJobCanceled; + } + + /** + * getter + * @return average job run time. + */ + public final double getAvgJobRunTime() { + return avgJobRunTime; + } + + /** + * getter + * @return stand. dev. of run time. + */ + public final double getStdJobRunTime() { + return stdJobRunTime; + } + + /** + * getter + * @return last teim (epoch ms) job of this class started. + */ + public final long getLastTimeJobStarted() { + return lastTimeJobStarted; + } + + /** + * getter + * @return true if retrying job. + */ + public final boolean isRetryJob() { + return retryJob; + } + + /** + * getter + * @return true if logging job. + */ + public final boolean isLoggingJob() { + return loggingJob; + } + + /** + * getter + * @return true if is resource throttled job. + */ + public final boolean isResourceThrottledJob() { + return resourceThrottledJob; + } + + /** + * getter + * @return number of times job retried [if trying job]. + */ + public final long getNumRetryJobStarts() { + return numRetryJobStarts; + } + + /** + * getter + * @return number of time retry job was successful. + */ + public final long getNumRetryJobSuccessful() { + return numRetryJobSuccessful; + } + + /** + * getter + * @return number of times retry job failed. + */ + public final long getNumRetryJobFailures() { + return numRetryJobFailures; + } + + /** + * getter + * @return number of time retry job was cancelled. + */ + public final long getNumRetryJobCanceled() { + return numRetryJobCanceled; + } + + /** + * getter + * @return number of times job was retried. + */ + public final long getNumJobTries() { + return numJobTries; + } + + /** + * getter + * @return average number of times inner job retried. + */ + public final double getAvgNumJobTries() { + return avgNumJobTries; + } + + /** + * getter + * @return average runtime (ms) retry job ran. + */ + public final double getAvgRetryJobRunTime() { + return avgRetryJobRunTime; + } + + /** + * getter + * @return stand. dev. of retry job runtime. + */ + public final double getStdRetryJobRunTime() { + return stdRetryJobRunTime; + } + + /** + * getter + * @return total resource lock wait time in ms. [if resource throttled job] + */ + public final double getTotalResourceThrottledWaitTime() { + return totalResourceThrottledWaitTime; + } + + /** + * getter + * @return average resource wait time [if resource throttled job] + */ + public final double getAvgResourceThrottledWaitTime() { + return avgResourceThrottledWaitTime; + } + + /** + * getter + * @return stand. dev. of resource wait time [if resource throttled job] + */ + public final double getStdResourceThrottledWaitTime() { + return stdResourceThrottledWaitTime; + } + + /** + * getter + * @return num times resource throttled job started. + */ + public final long getNumResourceThrottledStart() { + return numResourceThrottledStarted; + } + + /** + * getter + * @return num times logging job starts. + */ + public final long getNumLoggingJobStarts() { + return numLoggingJobStarts; + } + + /** + * getter + * @return num time resource throttled job started. + */ + public final long getNumResourceThrottledStarted() { + return numResourceThrottledStarted; + } + + /** + * getter + * @return average job failed time (ms). + */ + public final double getAvgJobFailedTime() { + return avgJobFailedTime; + } + + /** + * getter + * @return stand. dev. fo job fail time (ms). + */ + public final double getStdJobFailedTime() { + return stdJobFailedTime; + } + + /** + * getter + * @return average run time on canceled jobs (ms). + */ + public final double getAvgJobCanceledTime() { + return avgJobCanceledTime; + } + + /** + * getter + * @return stand. dev. for run time on canceled jobs (ms). + */ + public final double getStdJobCanceledTime() { + return stdJobCanceledTime; + } +} diff --git a/azkaban/src/java/azkaban/monitor/stats/NativeWorkflowClassStats.java b/azkaban/src/java/azkaban/monitor/stats/NativeWorkflowClassStats.java new file mode 100644 index 0000000..a89681b --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/stats/NativeWorkflowClassStats.java @@ -0,0 +1,259 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.stats; + + +/** + * Base data-version of WorkflowClassStats comprising all data fields and getters. + * Includes constructor for building clone of current values. Used to acquire + * stable version of actual NativeWorkflowClassStats, for user. + * This class is not thread safe. + * + */ +public class NativeWorkflowClassStats extends ClassStats { + /** + * Name of the root job for this workflow. + */ + protected String workflowRootName; + + /** + * Number of times this workflow was scheduled. + */ + protected long numTimesWorkflowScheduled; + + /** + * Number of times this workflow was started. + */ + protected long numTimesWorkflowStarted; + + /** + * Number of times this workflow ended successfully. + */ + protected long numTimesWorkflowSuccessful; + + /** + * Number of times this workflow ended in failure. + */ + protected long numTimesWorkflowFailed; + + /** + * number of times this workflow was canceled. + */ + protected long numTimesWorkflowCanceled; + + /** + * Average runtime for this workflow successfulrun. Strictly start to end, + * does not include scheduled time. + */ + protected double avgWorkflowRunTime; + + /** + * Standard deviation for workflow successful run time. + */ + protected double stdWorkflowRunTime; + + /** + * Average runtime for workflows that failed. + */ + protected double avgWorkflowFailedTime; + + /** + * Standard deviation of runtime for workflows that failed. + */ + protected double stdWorkflowFailedTime; + + /** + * Average runtime for workflows that were canceled. + */ + protected double avgWorkflowCanceledTime; + + /** + * Standard deviateion of runtime for workflows that are canceled. + */ + protected double stdWorkflowCanceledTime; + + /** + * Average time workflow spend in pending state. + */ + protected double avgWorkflowPendingTime; + + /** + * Standard deviation for workflow pending. + */ + protected double stdWorkflowPendingTime; + + /** + * Most recent time this workflow was started. + */ + protected long lastWorkflowTimeStarted; + + /** + * Constructor used to build the workflow class + */ + public NativeWorkflowClassStats() { + super(); + } + + /** + * Copy constructor to construct duplicate based on what is in the master copy. + * Includes only the critical statistics. + * @param nwcs + * NativeWorkflowClassStats being copied. + */ + public NativeWorkflowClassStats(NativeWorkflowClassStats nwcs) { + super(); // Do not pass monitor to copies. + synchronized (nwcs) { + this.workflowRootName = nwcs.workflowRootName; + this.numTimesWorkflowScheduled = nwcs.numTimesWorkflowScheduled; + this.numTimesWorkflowStarted = nwcs.numTimesWorkflowStarted; + this.numTimesWorkflowSuccessful = nwcs.numTimesWorkflowSuccessful; + this.numTimesWorkflowFailed = nwcs.numTimesWorkflowFailed; + this.numTimesWorkflowCanceled = nwcs.numTimesWorkflowCanceled; + this.avgWorkflowRunTime = nwcs.avgWorkflowRunTime; + this.stdWorkflowRunTime = nwcs.stdWorkflowRunTime; + this.avgWorkflowFailedTime = nwcs.avgWorkflowFailedTime; + this.stdWorkflowFailedTime = nwcs.stdWorkflowFailedTime; + this.avgWorkflowCanceledTime = nwcs.stdWorkflowCanceledTime; + this.avgWorkflowPendingTime = nwcs.avgWorkflowPendingTime; + this.stdWorkflowPendingTime = nwcs.stdWorkflowPendingTime; + this.lastWorkflowTimeStarted = nwcs.lastWorkflowTimeStarted; + } + } + + @Override + public final Object clone() throws CloneNotSupportedException { + throw new CloneNotSupportedException(); + } + + /** + * getter + * @return name of root job. + */ + public final String getWorkflowRootName() { + return workflowRootName; + } + + /** + * getter + * @return number of times workflow scheduled. + */ + public final long getNumTimesWorkflowScheduled() { + return numTimesWorkflowScheduled; + } + + /** + * getter + * @return number of times workflow started. + */ + public final long getNumTimesWorkflowStarted() { + return numTimesWorkflowStarted; + } + + /** + * getter + * @return number of times workflow was successful. + */ + public final long getNumTimesWorkflowSuccessful() { + return numTimesWorkflowSuccessful; + } + + /** + * getter + * @return number of times workflow failed. + */ + public final long getNumTimesWorkflowFailed() { + return numTimesWorkflowFailed; + } + + /** + * getter + * @return number of time workflow canceled. + */ + public final long getNumTimesWorkflowCanceled() { + return numTimesWorkflowCanceled; + } + + /** + * getter + * @return average workflow run time in ms. + */ + public final double getAvgWorkflowRunTime() { + return avgWorkflowRunTime; + } + + /** + * getter + * @return stand. dev. for workflow run time. + */ + public final double getStdWorkflowRunTime() { + return stdWorkflowRunTime; + } + + /** + * getter + * @return last time workflow of this class was started. + */ + public final long getLastWorkflowTimeStarted() { + return lastWorkflowTimeStarted; + } + + /** + * getter + * @return stand. dev. for pending time (ms). + */ + public final double getStdWorkflowPendingTime() { + return stdWorkflowPendingTime; + } + + /** + * getter + * @return average workflow pending time (scheduled but not started) ms. + */ + public final double getAvgWorkflowPendingTime() { + return avgWorkflowPendingTime; + } + + /** + * getter + * @return average workflow runtime for failed workflows. + */ + public final double getAvgWorkflowFailedTime() { + return avgWorkflowFailedTime; + } + + /** + * getter + * @return stand. dev. on runtime for failed workflows (ms). + */ + public final double getStdWorkflowFailedTime() { + return stdWorkflowFailedTime; + } + + /** + * getter + * @return average canceled workflow runtime. + */ + public final double getAvgWorkflowCanceledTime() { + return avgWorkflowCanceledTime; + } + + /** + * getter + * @return stand. dev on runtime of canceled workflows. + */ + public final double getStdWorkflowCanceledTime() { + return stdWorkflowCanceledTime; + } +} diff --git a/azkaban/src/java/azkaban/monitor/stats/WorkflowClassStats.java b/azkaban/src/java/azkaban/monitor/stats/WorkflowClassStats.java new file mode 100644 index 0000000..7ee562a --- /dev/null +++ b/azkaban/src/java/azkaban/monitor/stats/WorkflowClassStats.java @@ -0,0 +1,130 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor.stats; + +import org.apache.commons.math.stat.descriptive.SynchronizedSummaryStatistics; + +import azkaban.monitor.model.WorkflowExecutionModel; + +/** + * For a given job root name, keep track of statistics for all workflows that + * originate from jobs with that name. + */ +public class WorkflowClassStats extends NativeWorkflowClassStats { + + /** + * Math structures to compute averages and stdvar.for pending. + */ + private SynchronizedSummaryStatistics normalWfPendingStats = new SynchronizedSummaryStatistics(); + + /** + * Math structures to compute averages and stdvar. for runtime. + */ + private SynchronizedSummaryStatistics normalWfRunStats = new SynchronizedSummaryStatistics(); + + /** + * Math structures to compute averages and stdvar. for runtimes of failed workflows. + */ + private SynchronizedSummaryStatistics failedWfRunStats = new SynchronizedSummaryStatistics(); + + /** + * Math structures to compute averages and stdvar. for runtimes of canceled workflows. + */ + private SynchronizedSummaryStatistics canceledWfRunStats = new SynchronizedSummaryStatistics(); + + /** + * Constructor used to build the workflow class + * @param workflowRootName + * workflow root job name + */ + public WorkflowClassStats(String workflowRootName) { + super(); + this.workflowRootName = workflowRootName; + } + + /** + * Update statistics based on the scheduling of a workflow. + * @param wfModel + * workflow execution behind scheduling stats. + */ + public synchronized void updateWorkflowScheduledStats(WorkflowExecutionModel wfModel) { + numTimesWorkflowScheduled++; + } + + /** + * Update statistics based on the starting of a workflow. + * @param wfModel + * workflow exection behind starting stats. + */ + public synchronized void updateWorkflowStartedStats(WorkflowExecutionModel wfModel) { + numTimesWorkflowStarted++; + if (lastWorkflowTimeStarted == 0 || + lastWorkflowTimeStarted < wfModel.getStartTime()) { + lastWorkflowTimeStarted = wfModel.getStartTime(); + } + + long incTime = wfModel.getPendingTime(); + normalWfPendingStats.addValue(incTime); + avgWorkflowPendingTime = normalWfPendingStats.getMean(); + stdWorkflowPendingTime = normalWfPendingStats.getStandardDeviation(); + } + + /** + * Update workflow statistics based on the ending of a workflow. + * @param wfModel + * the workflow execution behind workflow ending stats. + */ + public synchronized void updateWorkflowEndedStats(WorkflowExecutionModel wfModel) { + SynchronizedSummaryStatistics wfStats = null; + switch (wfModel.getFinalWorkflowState()) { + case SUCCESSFUL: + numTimesWorkflowSuccessful++; + wfStats = normalWfRunStats; + break; + case FAILED: + numTimesWorkflowFailed++; + wfStats = failedWfRunStats; + break; + case CANCELED: + numTimesWorkflowCanceled++; + wfStats = canceledWfRunStats; + break; + default: + return; + } + + /** + * We are only keeping averages and std dev on successful workflows. + */ + long incTime = wfModel.getEndTime() - wfModel.getStartTime(); + wfStats.addValue(incTime); + double avg = wfStats.getMean(); + double std = wfStats.getStandardDeviation(); + switch(wfModel.getFinalWorkflowState()) { + case SUCCESSFUL: + avgWorkflowRunTime = avg; + stdWorkflowRunTime = std; + break; + case FAILED: + avgWorkflowFailedTime = avg; + stdWorkflowFailedTime = std; + break; + case CANCELED: + avgWorkflowCanceledTime = avg; + stdWorkflowCanceledTime = std; + break; + } + } +} diff --git a/azkaban/src/java/azkaban/scheduler/ScheduleManager.java b/azkaban/src/java/azkaban/scheduler/ScheduleManager.java index 41054e9..6fc674f 100644 --- a/azkaban/src/java/azkaban/scheduler/ScheduleManager.java +++ b/azkaban/src/java/azkaban/scheduler/ScheduleManager.java @@ -17,6 +17,9 @@ import azkaban.jobs.JobExecutionException; import azkaban.jobs.JobExecutorManager; +import azkaban.monitor.MonitorImpl; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.MonitorInternalInterface.WorkflowAction; /** @@ -186,6 +189,12 @@ public synchronized List getSchedule() { public synchronized void addScheduledJob(ScheduledJob job) { logger.info("Adding " + job + " to schedule."); schedule.add(job); + MonitorImpl.getInternalMonitorInterface().workflowEvent(null, + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + job.getId()); + this.interrupt(); } @@ -197,6 +206,11 @@ public synchronized void addScheduledJob(ScheduledJob job) { public synchronized void removeScheduledJob(ScheduledJob job) { logger.info("Removing " + job + " from the schedule."); schedule.remove(job); + MonitorImpl.getInternalMonitorInterface().workflowEvent(null, + System.currentTimeMillis(), + WorkflowAction.UNSCHEDULE_WORKFLOW, + WorkflowState.NOP, + job.getId()); // Don't need to interrupt, because if this is originally on the top of the queue, // it'll just skip it. } diff --git a/azkaban/src/unit/azkaban/flow/IndividualJobExecutableFlowTest.java b/azkaban/src/unit/azkaban/flow/IndividualJobExecutableFlowTest.java index 51e3ea2..9f6ee25 100644 --- a/azkaban/src/unit/azkaban/flow/IndividualJobExecutableFlowTest.java +++ b/azkaban/src/unit/azkaban/flow/IndividualJobExecutableFlowTest.java @@ -4,6 +4,11 @@ import azkaban.common.jobs.Job; import azkaban.common.utils.Props; import azkaban.jobs.Status; +import azkaban.jobs.builtin.ProcessJob; +import azkaban.monitor.MonitorImpl; +import azkaban.monitor.MonitorInternalInterface; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.MonitorInternalInterface.WorkflowAction; import org.easymock.IAnswer; import org.easymock.classextension.EasyMock; @@ -30,6 +35,7 @@ public class IndividualJobExecutableFlowTest private volatile AtomicBoolean assertionViolated; private volatile String reason; + private volatile MonitorImpl monitor; private static Throwable theException; private static Map theExceptions; @@ -50,6 +56,8 @@ public void setUp() assertionViolated = new AtomicBoolean(false); reason = "Default Reason"; + + monitor = (MonitorImpl)MonitorImpl.getMonitor(); } @After @@ -57,6 +65,8 @@ public void tearDown() { Assert.assertFalse(reason, assertionViolated.get()); EasyMock.verify(jobManager); + + MonitorImpl.unsetMonitor(); } @@ -70,7 +80,13 @@ public void testSanity() throws Throwable final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("blah", "blah", jobManager); EasyMock.expect(jobManager.loadJob("blah", overrideProps, true)).andReturn(mockJob).once(); - EasyMock.expect(mockJob.getId()).andReturn("success Job").once(); + EasyMock.expect(mockJob.getId()).andReturn("success Job").anyTimes(); + + monitor.workflowEvent("blah", + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + "successJob"); mockJob.run(); EasyMock.expectLastCall().andAnswer(new IAnswer() @@ -157,13 +173,22 @@ public void testFailure() throws Throwable final CountDownLatch completionLatch = new CountDownLatch(1); - final Job mockJob = EasyMock.createMock(Job.class); + final ProcessJob mockJob = EasyMock.createMock(ProcessJob.class); + final Props props = new Props(); + props.put("azkaban.flow.id", "23"); + final Props overrideProps = new Props(); final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("blah", "blah", jobManager); EasyMock.expect(jobManager.loadJob("blah", overrideProps, true)).andReturn(mockJob).once(); - EasyMock.expect(mockJob.getId()).andReturn("blah").times(1); + EasyMock.expect(mockJob.getId()).andReturn("failure Job").anyTimes(); + EasyMock.expect(mockJob.getProps()).andReturn(props).anyTimes(); + monitor.workflowEvent("23", + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + "failureJob"); mockJob.run(); EasyMock.expectLastCall().andAnswer(new IAnswer() @@ -203,7 +228,7 @@ public void completed(Status status) } }); - completionLatch.await(1000, TimeUnit.MILLISECONDS); + completionLatch.await(4000, TimeUnit.MILLISECONDS); Assert.assertEquals(Status.FAILED, executableFlow.getStatus()); Assert.assertEquals(theExceptions, executableFlow.getExceptions()); @@ -232,12 +257,21 @@ public void testAllExecuteCallbacksCalledOnSuccess() throws Throwable final CountDownLatch firstCallbackLatch = new CountDownLatch(1); final CountDownLatch secondCallbackLatch = new CountDownLatch(1); - final Job mockJob = EasyMock.createMock(Job.class); + final ProcessJob mockJob = EasyMock.createMock(ProcessJob.class); + final Props props = new Props(); + props.put("azkaban.flow.id", "23"); final Props overrideProps = new Props(); - final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("blah", "blah", jobManager); + final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("23", "blah", jobManager); EasyMock.expect(jobManager.loadJob("blah", overrideProps, true)).andReturn(mockJob).once(); - EasyMock.expect(mockJob.getId()).andReturn("success Job").once(); + EasyMock.expect(mockJob.getId()).andReturn("success Job").anyTimes(); + EasyMock.expect(mockJob.getProps()).andReturn(props).anyTimes(); + + monitor.workflowEvent("23", + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + "successJob"); mockJob.run(); EasyMock.expectLastCall().andAnswer(new IAnswer() @@ -314,13 +348,22 @@ public void testAllExecuteCallbacksCalledOnFailure() throws Throwable final CountDownLatch firstCallbackLatch = new CountDownLatch(1); final CountDownLatch secondCallbackLatch = new CountDownLatch(1); - final Job mockJob = EasyMock.createMock(Job.class); + final ProcessJob mockJob = EasyMock.createMock(ProcessJob.class); + final Props props = new Props(); + props.put("azkaban.flow.id", "23"); final Props overrideProps = new Props(); - final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("blah", "blah", jobManager); + final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("23", "blah", jobManager); EasyMock.expect(jobManager.loadJob("blah", overrideProps, true)).andReturn(mockJob).once(); - EasyMock.expect(mockJob.getId()).andReturn("blah").times(1); - + EasyMock.expect(mockJob.getId()).andReturn("success Job").anyTimes(); + EasyMock.expect(mockJob.getProps()).andReturn(props).anyTimes(); + + monitor.workflowEvent("23", + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + "successJob"); + mockJob.run(); EasyMock.expectLastCall().andThrow(theException).once(); @@ -360,8 +403,8 @@ public void theCallback(Status status) } }); - firstCallbackLatch.await(1000, TimeUnit.MILLISECONDS); - secondCallbackLatch.await(1000, TimeUnit.MILLISECONDS); + firstCallbackLatch.await(5000, TimeUnit.MILLISECONDS); + secondCallbackLatch.await(5000, TimeUnit.MILLISECONDS); Assert.assertEquals(Status.FAILED, executableFlow.getStatus()); Assert.assertEquals(theExceptions, executableFlow.getExceptions()); @@ -381,16 +424,25 @@ public void testReset() throws Exception { final CountDownLatch completionLatch = new CountDownLatch(1); - final Job mockJob = EasyMock.createMock(Job.class); + final ProcessJob mockJob = EasyMock.createMock(ProcessJob.class); + final Props props = new Props(); + props.put("azkaban.flow.id", "23"); final Props overrideProps = new Props(); - final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("blah", "blah", jobManager); + final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("23", "blah", jobManager); final Props firstProps = new Props(); final Props secondProps = new Props(); EasyMock.expect(jobManager.loadJob("blah", overrideProps, true)).andReturn(mockJob).once(); - EasyMock.expect(mockJob.getId()).andReturn("success Job").once(); - EasyMock.expect(mockJob.getJobGeneratedProperties()).andReturn(firstProps).once(); + EasyMock.expect(mockJob.getId()).andReturn("success Job").anyTimes(); + EasyMock.expect(mockJob.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(mockJob.getJobGeneratedProperties()).andReturn(firstProps).anyTimes(); + + monitor.workflowEvent("23", + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + "successJob"); mockJob.run(); EasyMock.expectLastCall().andAnswer(new IAnswer() @@ -430,7 +482,7 @@ public void completed(Status status) } }); - completionLatch.await(1000, TimeUnit.MILLISECONDS); + completionLatch.await(3000, TimeUnit.MILLISECONDS); Assert.assertEquals(Status.SUCCEEDED, executableFlow.getStatus()); Assert.assertEquals(emptyExceptions, executableFlow.getExceptions()); Assert.assertEquals(firstProps, executableFlow.getReturnProps()); @@ -446,7 +498,8 @@ public void completed(Status status) Assert.assertEquals(null, executableFlow.getReturnProps()); EasyMock.expect(jobManager.loadJob("blah", overrideProps, true)).andReturn(mockJob).once(); - EasyMock.expect(mockJob.getId()).andReturn("success Job").once(); + EasyMock.expect(mockJob.getId()).andReturn("success Job").anyTimes(); + EasyMock.expect(mockJob.getProps()).andReturn(props).anyTimes(); EasyMock.expect(mockJob.getJobGeneratedProperties()).andReturn(secondProps).once(); mockJob.run(); @@ -501,15 +554,24 @@ public void testResetWithFailedJob() throws Exception { final CountDownLatch completionLatch = new CountDownLatch(1); - final Job mockJob = EasyMock.createMock(Job.class); + final ProcessJob mockJob = EasyMock.createMock(ProcessJob.class); + final Props props = new Props(); + props.put("azkaban.flow.id", "23"); final Props overrideProps = new Props(); - final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("blah", "blah", jobManager); + final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("23", "blah", jobManager); executableFlow.setStatus(Status.FAILED); Assert.assertTrue("Should be able to reset the flow.", executableFlow.reset()); EasyMock.expect(jobManager.loadJob("blah", overrideProps, true)).andReturn(mockJob).once(); - EasyMock.expect(mockJob.getId()).andReturn("success Job").once(); + EasyMock.expect(mockJob.getId()).andReturn("success Job").anyTimes(); + EasyMock.expect(mockJob.getProps()).andReturn(props).anyTimes(); + + monitor.workflowEvent("23", + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + "successJob"); mockJob.run(); EasyMock.expectLastCall().andAnswer(new IAnswer() @@ -551,7 +613,7 @@ public void completed(Status status) } }); - completionLatch.await(1000, TimeUnit.MILLISECONDS); + completionLatch.await(3000, TimeUnit.MILLISECONDS); Assert.assertEquals(Status.SUCCEEDED, executableFlow.getStatus()); EasyMock.verify(mockJob); @@ -563,14 +625,16 @@ public void testCancel() throws Exception final CountDownLatch cancelLatch = new CountDownLatch(1); final CountDownLatch runLatch = new CountDownLatch(1); - final Job mockJob = EasyMock.createMock(Job.class); + final ProcessJob mockJob = EasyMock.createMock(ProcessJob.class); + final Props props = new Props(); + props.put("azkaban.flow.id", "23"); final Props overrideProps = new Props(); - final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("blah", "blah", jobManager); + final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow("23", "blah", jobManager); Assert.assertTrue("Should be able to reset the flow.", executableFlow.reset()); - EasyMock.expect(mockJob.getId()).andReturn("blah").once(); - mockJob.run(); + EasyMock.expect(mockJob.getId()).andReturn("blah").anyTimes(); + EasyMock.expect(mockJob.getProps()).andReturn(props).anyTimes(); mockJob.run(); EasyMock.expect(jobManager.loadJob("blah", overrideProps, true)).andAnswer(new IAnswer() { @Override @@ -583,7 +647,7 @@ public Job answer() throws Throwable } }).once(); - EasyMock.expect(mockJob.getJobGeneratedProperties()).andReturn(new Props()).once(); + EasyMock.expect(mockJob.getJobGeneratedProperties()).andReturn(new Props()).anyTimes(); EasyMock.replay(mockJob, jobManager); diff --git a/azkaban/src/unit/azkaban/jobs/builtin/WordCountLocal.java b/azkaban/src/unit/azkaban/jobs/builtin/WordCountLocal.java index 6b51c3d..bd8c6a5 100644 --- a/azkaban/src/unit/azkaban/jobs/builtin/WordCountLocal.java +++ b/azkaban/src/unit/azkaban/jobs/builtin/WordCountLocal.java @@ -66,6 +66,12 @@ public Props getJobGeneratedProperties() { return new Props(); } + + @Override + public boolean isCanceled() + { + return false; + } } diff --git a/azkaban/src/unit/azkaban/monitor/ExecutionModelTest.java b/azkaban/src/unit/azkaban/monitor/ExecutionModelTest.java new file mode 100644 index 0000000..5fde194 --- /dev/null +++ b/azkaban/src/unit/azkaban/monitor/ExecutionModelTest.java @@ -0,0 +1,479 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import java.util.List; +import java.util.Map; + +import junit.framework.Assert; + +import org.apache.log4j.Logger; +import org.easymock.classextension.EasyMock; +import org.joda.time.DateTimeUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import azkaban.common.utils.Props; +import azkaban.jobs.builtin.ProcessJob; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.model.ExecutionModel; +import azkaban.monitor.model.ExecutionModelImpl; +import azkaban.monitor.model.JobExecutionModel; +import azkaban.monitor.model.WorkflowExecutionModel; +import azkaban.monitor.stats.NativeGlobalStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +/** + * Class to test execution model functionality. + * @author donpazel + * + */ +public class ExecutionModelTest { + /** + * Logger used for this file. + */ + private static final Logger LOG = Logger.getLogger(ExecutionModelTest.class); + + /** + * Setup test method. + * @throws Exception + * Exception thrown + */ + @Before + public void setUp() throws Exception { } + + /** + * Teardown test method. + * @throws Exception + * Exception thown + */ + @After + public void tearDown() throws Exception { } + + /** + * Test for workflow of a single job. + * @throws Exception + * Exception thrown + */ + @Test + public void testWorkflowSingleJob() throws Exception { + String wfId = "23"; + String rootJobName = "rootJob"; + long schedTime = DateTimeUtils.currentTimeMillis(); + long increment = 1 * 1000; + long incCount = 0; + MonitorImpl aImpl = EasyMock.createStrictMock(MonitorImpl.class); + + ExecutionModel eModel = new ExecutionModelImpl(aImpl); + + LOG.info("Scheduling a workflow"); + eModel.scheduleWorkflow(rootJobName, schedTime + (incCount++) * increment ); + + NativeWorkflowClassStats nativeWf = eModel.getWorkflowClassStatsById(rootJobName); + Assert.assertNotNull(nativeWf); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowScheduled()); + + LOG.info("Starting a workflow"); + eModel.startWorkflow(rootJobName, wfId, schedTime + (incCount++) * increment); + + nativeWf = eModel.getWorkflowClassStatsById(rootJobName); + Assert.assertNotNull(nativeWf); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowScheduled()); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowStarted()); + + // Start a fictitious job. + String job1Name = "Job1"; + ProcessJob job1 = EasyMock.createStrictMock(ProcessJob.class); + Props job1Props = new Props(); + job1Props.put("azkaban.flow.id", wfId); + + EasyMock.expect(job1.getProps()).andReturn(job1Props).anyTimes(); + EasyMock.expect(job1.getId()).andReturn(job1Name).anyTimes(); + EasyMock.replay(job1); + + LOG.info("Starting a workflow job"); + eModel.startWorkflowJob(schedTime + (incCount++) * increment, job1); + NativeJobClassStats job1Stats = eModel.getJobClassStatsById(job1Name); + Assert.assertNotNull(job1Stats); + Assert.assertEquals(1, job1Stats.getNumTimesJobStarted()); + + LOG.info("Ending a workflow job"); + eModel.endWorkflowJob(schedTime + (incCount++) * increment, job1, JobState.SUCCESSFUL); + job1Stats = eModel.getJobClassStatsById(job1Name); + Assert.assertNotNull(job1Stats); + Assert.assertEquals(1, job1Stats.getNumTimesJobStarted()); + Assert.assertEquals(1, job1Stats.getNumTimesJobSuccessful()); + + LOG.info("Ending a workflow"); + eModel.endWorkflow(wfId, schedTime + (incCount++) * increment, MonitorInterface.WorkflowState.SUCCESSFUL); + + nativeWf = eModel.getWorkflowClassStatsById(rootJobName); + Assert.assertNotNull(nativeWf); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowScheduled()); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowStarted()); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowSuccessful()); + + Map allWfStats = eModel.getAllWorkflowClassStats(); + List allWfNames = eModel.getWorkflowClassIds(); + Assert.assertNotNull(allWfStats); + Assert.assertEquals(allWfStats.size(), allWfNames.size()); + Assert.assertEquals(allWfNames.size(), eModel.getNumberOfWorkflows()); + Assert.assertEquals(1, eModel.getNumberOfCompletedWorkflows()); + + NativeGlobalStats gs = eModel.getGlobalStatsCopy(); + Assert.assertEquals(1, gs.getTotalJobsStarted()); + Assert.assertEquals(1, gs.getTotalJobsSuccessful()); + Assert.assertEquals(1, gs.getTotalWorkflowsStarted()); + Assert.assertEquals(1, gs.getTotalWorkflowsSuccessful()); + + List jobIds = eModel.getJobClassIds(); + Assert.assertNotNull(jobIds); + Assert.assertEquals(1, jobIds.size()); + Assert.assertEquals(job1Name, jobIds.get(0)); + + NativeJobClassStats nativeJob = eModel.getJobClassStatsById(jobIds.get(0)); + Assert.assertNotNull(nativeJob); + Assert.assertEquals(1, nativeJob.getNumTimesJobStarted()); + Assert.assertEquals(1, nativeJob.getNumTimesJobSuccessful()); + + Map allJobStats = eModel.getAllJobClassStats(); + Assert.assertNotNull(allJobStats); + Assert.assertEquals(allJobStats.size(), jobIds.size()); + + List allWfModels = eModel.getCompletedWorkflowModels(schedTime + (incCount++) * increment, true); + Assert.assertNotNull(allWfModels); + Assert.assertEquals(1, allWfModels.size()); + WorkflowExecutionModel wfExecModel = allWfModels.get(0); + Assert.assertNotNull(wfExecModel); + long wfRunTime = wfExecModel.getExecutionTime(); + Assert.assertEquals(3000, wfRunTime); + long wfElapsedTime = wfExecModel.getElapsedTime(); + Assert.assertEquals(4000, wfElapsedTime); + long wfPendingTime = wfExecModel.getPendingTime(); + Assert.assertEquals(1000, wfPendingTime); + Assert.assertTrue(wfExecModel.isCompleted()); + List wfJobNames = wfExecModel.getWorkflowJobNames(); + Assert.assertEquals(1, wfJobNames.size()); + List wfJobExecs = wfExecModel.getWorkflowJobExecutions(); + Assert.assertNotNull(wfJobExecs); + Assert.assertEquals(1, wfJobExecs.size()); + JobExecutionModel wfJobModel = wfJobExecs.get(0); + Assert.assertNotNull(wfJobModel); + Assert.assertTrue(wfJobModel.isCompleted()); + long jobExecRunTime = wfJobModel.getExecutionTime(); + Assert.assertEquals(1000, jobExecRunTime); + Assert.assertEquals(wfExecModel, wfJobModel.getWorkflowModel()); + + eModel.clearCompletedWorkflows(); + allWfModels = eModel.getCompletedWorkflowModels(true); + Assert.assertNotNull(allWfModels); + Assert.assertEquals(0, allWfModels.size()); + + LOG.info("End testWorkflowSingleJob"); + } + + /** + * Test with multiple workflows. + * @throws Exception + * Exception thrown + */ + @Test + public void testWorkflowMultiJob() throws Exception { + String wfId = "32"; + String rootJobName = "rootJob"; + long schedTime = DateTimeUtils.currentTimeMillis(); + long increment = 1 * 1000; + long incCount = 0; + MonitorImpl aImpl = EasyMock.createStrictMock(MonitorImpl.class); + + ExecutionModel eModel = new ExecutionModelImpl(aImpl); + + LOG.info("Scheduling a workflow"); + eModel.scheduleWorkflow(rootJobName, schedTime + (incCount++) * increment); + + NativeWorkflowClassStats nativeWf = eModel.getWorkflowClassStatsById(rootJobName); + Assert.assertNotNull(nativeWf); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowScheduled()); + + LOG.info("Starting a workflow"); + eModel.startWorkflow(rootJobName, wfId, schedTime + (incCount++) * increment); + + nativeWf = eModel.getWorkflowClassStatsById(rootJobName); + Assert.assertNotNull(nativeWf); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowScheduled()); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowStarted()); + + for (int i = 1; i <= 10; ++i) { + String jobName = "Job" + i; + ProcessJob job = EasyMock.createStrictMock(ProcessJob.class); + Props jobProps = new Props(); + jobProps.put("azkaban.flow.id", wfId); + + EasyMock.expect(job.getProps()).andReturn(jobProps).anyTimes(); + EasyMock.expect(job.getId()).andReturn(jobName).anyTimes(); + EasyMock.replay(job); + + LOG.info("Starting a workflow job"); + eModel.startWorkflowJob(schedTime + (incCount++) * increment, job); + NativeJobClassStats jobStats = eModel.getJobClassStatsById(jobName); + Assert.assertNotNull(jobStats); + Assert.assertEquals(1, jobStats.getNumTimesJobStarted()); + + LOG.info("Ending a workflow job"); + JobState jState = JobState.SUCCESSFUL; + if (i == 5) { + jState = JobState.FAILED; + } else if (i == 7) { + jState = JobState.CANCELED; + } + eModel.endWorkflowJob(schedTime + (incCount++) * increment, job, jState); + jobStats = eModel.getJobClassStatsById(jobName); + Assert.assertNotNull(jobStats); + Assert.assertEquals(1, jobStats.getNumTimesJobStarted()); + if (i == 5) { + Assert.assertEquals(1, jobStats.getNumTimesJobFailed()); + } else if (i == 7) { + Assert.assertEquals(1, jobStats.getNumTimesJobCanceled()); + } else { + Assert.assertEquals(1, jobStats.getNumTimesJobSuccessful()); + } + } + + LOG.info("Ending a workflow"); + eModel.endWorkflow(wfId, + schedTime + (incCount++) * increment, + MonitorInterface.WorkflowState.SUCCESSFUL); + + nativeWf = eModel.getWorkflowClassStatsById(rootJobName); + Assert.assertNotNull(nativeWf); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowScheduled()); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowStarted()); + Assert.assertEquals(1, nativeWf.getNumTimesWorkflowSuccessful()); + + NativeGlobalStats nativeGs = eModel.getGlobalStatsCopy(); + Assert.assertNotNull(nativeGs); + Assert.assertEquals(10, nativeGs.getTotalJobsStarted()); + Assert.assertEquals(8, nativeGs.getTotalJobsSuccessful()); + Assert.assertEquals(1, nativeGs.getTotalJobsFailed()); + Assert.assertEquals(1, nativeGs.getTotalJobsCanceled()); + + + LOG.info("End testWorkflowMultiJob"); + } + + /** + * Test multiple workflows and job classes. + * @throws Exception + * Exception thrown + */ + @Test + public void testMultiWorkflowMultiJob() throws Exception { + String rootJobName = "rootJob"; + long schedTime = DateTimeUtils.currentTimeMillis(); + long increment = 1 * 1000; + MonitorImpl aImpl = EasyMock.createStrictMock(MonitorImpl.class); + + ExecutionModel eModel = new ExecutionModelImpl(aImpl); + + long totalWfTime = 0; + long totalWfPendingTime = 0; + for (int i = 58; i < 68; ++i) { + String wfId = Integer.toString(i); + LOG.info("Scheduling workflow " + i); + long wfPendingStartTime = schedTime; + eModel.scheduleWorkflow(rootJobName, schedTime); + schedTime += increment; + + LOG.info("Starting a workflow"); + totalWfPendingTime += schedTime - wfPendingStartTime; + long wfStartTime = schedTime; + eModel.startWorkflow(rootJobName, wfId, schedTime); + schedTime += increment; + + for (int j = 1; j <= 10; ++j) { + String jobName = "Job" + j; + ProcessJob job = EasyMock.createStrictMock(ProcessJob.class); + Props jobProps = new Props(); + jobProps.put("azkaban.flow.id", wfId); + + EasyMock.expect(job.getProps()).andReturn(jobProps).anyTimes(); + EasyMock.expect(job.getId()).andReturn(jobName).anyTimes(); + EasyMock.replay(job); + + LOG.info("Starting a workflow job"); + eModel.startWorkflowJob(schedTime, job); + schedTime += increment + Math.random() * 10.0; + + LOG.info("Ending a workflow job"); + JobState jState = JobState.SUCCESSFUL; + if (j == 5) { + jState = JobState.FAILED; + } else if (j == 7) { + jState = JobState.CANCELED; + } + eModel.endWorkflowJob(schedTime, job, jState); + schedTime += increment; + } + + LOG.info("Ending workflow " + i); + WorkflowState wState = WorkflowState.SUCCESSFUL; + if (i == 58) { + wState = WorkflowState.FAILED; + } else if (i == 59) { + wState = WorkflowState.CANCELED; + } + eModel.endWorkflow(wfId, + schedTime, + wState); + if (i != 58 && i != 59) { + totalWfTime += (schedTime - wfStartTime); + } + schedTime += increment; + } + + NativeJobClassStats job5Stats = eModel.getJobClassStatsById("Job5"); + Assert.assertNotNull(job5Stats); + Assert.assertEquals(10, job5Stats.getNumTimesJobStarted()); + Assert.assertEquals(10, job5Stats.getNumTimesJobFailed()); + Assert.assertEquals(1000, job5Stats.getAvgJobFailedTime(), 20.0); + Assert.assertEquals(5.0, job5Stats.getStdJobFailedTime(), 5.0); + + NativeJobClassStats job7Stats = eModel.getJobClassStatsById("Job7"); + Assert.assertNotNull(job7Stats); + Assert.assertEquals(10, job7Stats.getNumTimesJobStarted()); + Assert.assertEquals(10, job7Stats.getNumTimesJobCanceled()); + Assert.assertEquals(1000, job7Stats.getAvgJobCanceledTime(), 20.0); + Assert.assertEquals(5.0, job7Stats.getStdJobCanceledTime(), 5.0); + + NativeJobClassStats job9Stats = eModel.getJobClassStatsById("Job9"); + Assert.assertNotNull(job9Stats); + Assert.assertEquals(10, job9Stats.getNumTimesJobStarted()); + Assert.assertEquals(10, job9Stats.getNumTimesJobSuccessful()); + Assert.assertEquals(1000, job9Stats.getAvgJobRunTime(), 20.0); + Assert.assertEquals(5.0, job9Stats.getStdJobRunTime(), 5.0); + + List wfClassList = eModel.getWorkflowClassIds(); + Assert.assertNotNull(wfClassList); + Assert.assertEquals(1, wfClassList.size()); + Assert.assertEquals(rootJobName, wfClassList.get(0)); + NativeWorkflowClassStats wfStats = eModel.getWorkflowClassStatsById(rootJobName); + Assert.assertNotNull(wfStats); + Assert.assertEquals(10, wfStats.getNumTimesWorkflowScheduled()); + Assert.assertEquals(10, wfStats.getNumTimesWorkflowStarted()); + Assert.assertEquals(8, wfStats.getNumTimesWorkflowSuccessful()); + Assert.assertEquals(1, wfStats.getNumTimesWorkflowFailed()); + Assert.assertEquals(1, wfStats.getNumTimesWorkflowCanceled()); + Assert.assertEquals(wfStats.getAvgWorkflowRunTime(), totalWfTime / 8.0, 1.0); + Assert.assertEquals(wfStats.getAvgWorkflowPendingTime(), totalWfPendingTime / 10.0, 1.0); + + LOG.info("End testMultiWorkflowMultiJob"); + } + + /** + * Test multiple workflows, testing workflow models. + * @throws Exception + * Exception thrown + */ + @Test + public void testMultiWorkflowSameWfid() throws Exception { + String wfId = "89"; + String rootJobName = "rootJob"; + long schedTime = DateTimeUtils.currentTimeMillis(); + long increment = 1 * 1000; + long incCount = 0; + MonitorImpl aImpl = EasyMock.createStrictMock(MonitorImpl.class); + + ExecutionModel eModel = new ExecutionModelImpl(aImpl); + + LOG.info("Scheduling a workflow"); + eModel.scheduleWorkflow(rootJobName, schedTime + (incCount++) * increment); + + LOG.info("Starting a workflow"); + eModel.startWorkflow(rootJobName, wfId, schedTime + (incCount++) * increment); + + LOG.info("Ending a workflow"); + eModel.endWorkflow(wfId, + schedTime + (incCount++) * increment, + MonitorInterface.WorkflowState.SUCCESSFUL); + + List wfModelNames = eModel.getWorkflowClassIds(); + Assert.assertEquals(1, wfModelNames.size()); + List wfModelsPass1 = eModel.getCompletedWorkflowModels(schedTime + (incCount++) * increment, false); + Assert.assertEquals(1, wfModelsPass1.size()); + WorkflowExecutionModel olderWfModer = wfModelsPass1.get(0); + Assert.assertNotNull(olderWfModer); + + // Restart the workflow + LOG.info("Starting a workflow"); + eModel.startWorkflow(rootJobName, wfId, schedTime + (incCount++) * increment); + + LOG.info("Ending a workflow"); + eModel.endWorkflow(wfId, + schedTime + (incCount++) * increment, + MonitorInterface.WorkflowState.SUCCESSFUL); + + + // Explore how the workflow model structure. + List wfModelsPass2 = eModel.getCompletedWorkflowModels(schedTime + (incCount++) * increment, false); + Assert.assertEquals(1, wfModelsPass2.size()); + WorkflowExecutionModel newerWfModel = wfModelsPass2.get(0); + Assert.assertNotNull(newerWfModel); + Assert.assertEquals(olderWfModer, newerWfModel.getOlderExecutionModel()); + } + + /** + * Test clearing out completed workflows. + * @throws Exception + * Exception thrown. + */ + @Test + public void testClearWorkflowSameWfid() throws Exception { + String wfId = "89"; + String rootJobName = "rootJob"; + long schedTime = DateTimeUtils.currentTimeMillis() - 5 * 1000; + long increment = 1 * 1000; + long incCount = 0; + MonitorImpl aImpl = EasyMock.createStrictMock(MonitorImpl.class); + + ExecutionModel eModel = new ExecutionModelImpl(aImpl); + + LOG.info("Scheduling a workflow"); + eModel.scheduleWorkflow(rootJobName, schedTime); + + LOG.info("Starting a workflow"); + eModel.startWorkflow(rootJobName, wfId, schedTime + (incCount++) * increment); + + LOG.info("Ending a workflow"); + eModel.endWorkflow(wfId, + schedTime + (incCount++) * increment, + MonitorInterface.WorkflowState.SUCCESSFUL); + + List wfModelsPass1 = eModel.getCompletedWorkflowModels(schedTime, false); + Assert.assertEquals(0, wfModelsPass1.size()); + eModel.clearCompletedWorkflows(schedTime); + long numWorkflowsLeft = eModel.getNumberOfWorkflows(); + Assert.assertEquals(1, numWorkflowsLeft); + + List wfModelsPass2 = eModel.getCompletedWorkflowModels(false); + Assert.assertEquals(1, wfModelsPass2.size()); + eModel.clearCompletedWorkflows(); + numWorkflowsLeft = eModel.getNumberOfWorkflows(); + Assert.assertEquals(0, numWorkflowsLeft); + } + +} diff --git a/azkaban/src/unit/azkaban/monitor/FlowDirectoryTest.java b/azkaban/src/unit/azkaban/monitor/FlowDirectoryTest.java new file mode 100644 index 0000000..0992050 --- /dev/null +++ b/azkaban/src/unit/azkaban/monitor/FlowDirectoryTest.java @@ -0,0 +1,91 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import junit.framework.Assert; + +import org.apache.log4j.Logger; +import org.easymock.classextension.EasyMock; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import azkaban.app.LoggingJob; +import azkaban.common.utils.Props; +import azkaban.jobcontrol.impl.jobs.ResourceThrottledJob; +import azkaban.jobcontrol.impl.jobs.RetryingJob; +import azkaban.jobs.builtin.ProcessJob; +import azkaban.monitor.model.FlowDirectory; + +public class FlowDirectoryTest { + private static Logger logger = Logger.getLogger(FlowDirectoryTest.class); + + private FlowDirectory flowDirectory; + + @Before + public void setUp() throws Exception { + flowDirectory = FlowDirectory.getFlowDirectory(); + } + + @After + public void tearDown() throws Exception { + FlowDirectory.unsetFlowDirectory(); + } + + @Test + public void testFlowDirectory() throws Exception { + String wfId = "26"; + String pJobName = "pjobname"; + Props props = new Props(); + props.put("azkaban.flow.id", wfId); + + ProcessJob pJob = EasyMock.createStrictMock(ProcessJob.class); + ProcessJob p1Job = EasyMock.createStrictMock(ProcessJob.class); + + EasyMock.expect(pJob.getId()).andReturn(pJobName).anyTimes(); + EasyMock.expect(pJob.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(pJob.getId()).andReturn(pJobName).anyTimes(); + EasyMock.replay(pJob); + + EasyMock.expect(p1Job.getId()).andReturn(pJobName).anyTimes(); + EasyMock.expect(p1Job.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(p1Job.getId()).andReturn(pJobName).anyTimes(); + EasyMock.replay(p1Job); + + RetryingJob retryJob = new RetryingJob(pJob, 1, 1); + ResourceThrottledJob rtJob = new ResourceThrottledJob(retryJob, new TimeDelayJobLock(500L)); + LoggingJob lJob = new LoggingJob("/Users/root", rtJob, "finiky"); + + flowDirectory.mapJob(lJob); + flowDirectory.mapJob(p1Job); + + Assert.assertEquals(retryJob, flowDirectory.getDelegatingJobParent(pJob)); + Assert.assertEquals(rtJob, flowDirectory.getDelegatingJobParent(retryJob)); + Assert.assertEquals(lJob, flowDirectory.getDelegatingJobParent(rtJob)); + Assert.assertEquals(wfId, flowDirectory.getFlowId(pJob)); + Assert.assertEquals(wfId, flowDirectory.getFlowId(pJob)); + Assert.assertEquals(wfId, flowDirectory.getFlowId(retryJob)); + Assert.assertEquals(wfId, flowDirectory.getFlowId(rtJob)); + Assert.assertEquals(wfId, flowDirectory.getFlowId(lJob)); + Assert.assertEquals(wfId, flowDirectory.getFlowId(p1Job)); + + flowDirectory.removeJobReference(lJob); + Assert.assertNull(flowDirectory.getDelegatingJobParent(rtJob)); + Assert.assertNull(flowDirectory.getFlowId(lJob)); + Assert.assertEquals(rtJob, flowDirectory.getDelegatingJobParent(retryJob)); + + logger.info("End testFlowDirectory"); + } +} diff --git a/azkaban/src/unit/azkaban/monitor/LongArgJobMonitorTest.java b/azkaban/src/unit/azkaban/monitor/LongArgJobMonitorTest.java new file mode 100644 index 0000000..fea8edc --- /dev/null +++ b/azkaban/src/unit/azkaban/monitor/LongArgJobMonitorTest.java @@ -0,0 +1,327 @@ +/* + * Copyright (C) 2011 Adconion, Inc + * + * 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 azkaban.monitor; + +import java.io.IOException; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.log4j.Logger; +import org.easymock.classextension.EasyMock; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import azkaban.app.JobDescriptor; +import azkaban.app.JobManager; +import azkaban.common.utils.Props; +import azkaban.flow.FlowCallback; +import azkaban.flow.IndividualJobExecutableFlow; +import azkaban.jobs.AbstractProcessJob; +import azkaban.jobs.Status; +import azkaban.jobs.builtin.PythonJob; +import azkaban.jobs.builtin.Utils; +import azkaban.monitor.MonitorInterface.GlobalNotificationType; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.MonitorInternalInterface.WorkflowAction; +import azkaban.monitor.model.WorkflowExecutionModel; +import azkaban.monitor.stats.ClassStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +/** + * @author ibrahimulukaya + * Tests to check LongArgJob events in Monitor + */ +public class LongArgJobMonitorTest implements MonitorListener{ + private static Logger logger = Logger.getLogger(MonitorImplTest.class); + private volatile JobManager jobManager; + + private volatile AtomicBoolean assertionViolated; + private volatile String reason; + private volatile MonitorImpl monitor; + private static Map emptyExceptions; + + private final String wfId = "25"; + private final String rootJobName = "MyJob"; + + private int specificJobClassNotifications; + + private static final String scriptContentSuccess = + "#!/usr/bin/python \n" + + "import sys \n" + + "print sys.argv"; + + private static final String scriptContentFail = + "#!/usr/bin/python \n" + + "import sys \n" + + "print sys.argv \n" + + "sys.exit(1)"; + + private static String scriptFileSuccess ; + private static String scriptFileFail; + + @BeforeClass + public static void init() throws Exception { + emptyExceptions = new HashMap(); + long time = (new Date()).getTime(); + scriptFileSuccess = "/tmp/azkaban_python" + time + ".py"; + scriptFileFail = "/tmp/prop_print" + time + ".py"; + // dump script file + try { + Utils.dumpFile(scriptFileSuccess, scriptContentSuccess); + Utils.dumpFile(scriptFileFail, scriptContentFail); + } + catch (IOException e) { + e.printStackTrace(System.err); + Assert.fail("error in creating script file:" + e.getLocalizedMessage()); + } + } + + @AfterClass + public static void cleanup() { + // remove the input file and error input file + Utils.removeFile(scriptFileSuccess); + Utils.removeFile(scriptFileFail); + } + + @Before + public void setUp() + { + jobManager = EasyMock.createMock(JobManager.class); + + assertionViolated = new AtomicBoolean(false); + reason = "Default Reason"; + + monitor = (MonitorImpl)MonitorImpl.getMonitor(); + specificJobClassNotifications = 0; + } + + @After + public void tearDown() + { + monitor.deregisterJobClassNotification(this, rootJobName); + MonitorImpl.unsetMonitor(); + } + + @Test + public void testLongArgSuccess() throws Throwable + { + final CountDownLatch completionLatch = new CountDownLatch(1); + monitor.registerJobClassNotification(this, rootJobName); + + PythonJob pythonJob = null; + JobDescriptor descriptor = null; + Props props = null; + + descriptor = EasyMock.createMock(JobDescriptor.class); + + props = new Props(); + props.put("input_date", "20110505"); + props.put(AbstractProcessJob.WORKING_DIR, "."); + props.put("type", "python"); + props.put("script", scriptFileSuccess); + props.put("output_dir", "/data/report-${input_date}"); + props.put("azkaban.flow.id", wfId); + + EasyMock.expect(descriptor.getId()).andReturn(rootJobName).times(1); + EasyMock.expect(descriptor.getProps()).andReturn(props).times(3); + EasyMock.expect(descriptor.getFullPath()).andReturn(".").times(1); + + EasyMock.replay(descriptor); + + pythonJob = new PythonJob(descriptor); + EasyMock.verify(descriptor); + + final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow(wfId, rootJobName, jobManager); + + EasyMock.expect(jobManager.loadJob(rootJobName, props, true)).andReturn(pythonJob).once(); + + monitor.workflowEvent(null, + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + wfId); + + monitor.workflowEvent(wfId, + System.currentTimeMillis(), + WorkflowAction.START_WORKFLOW, + WorkflowState.NOP, + rootJobName); + + EasyMock.replay(jobManager); + Assert.assertEquals(Status.READY, executableFlow.getStatus()); + + executableFlow.execute( + props, + new FlowCallback() + { + @Override + public void progressMade() + { + } + + @Override + public void completed(Status status) + { + completionLatch.countDown(); + // Lazy failure is used since this doesn't run in the main thread + if (Status.SUCCEEDED != status) { + assertionViolated.set(true); + reason = String.format("In executableFlow Callback: status[%s] != Status.SUCCEEDED", status); + } + } + }); + + monitor.workflowEvent(wfId, + System.currentTimeMillis(), + WorkflowAction.END_WORKFLOW, + WorkflowState.SUCCESSFUL, + rootJobName); + + completionLatch.await(1000, TimeUnit.MILLISECONDS); + Assert.assertEquals(Status.SUCCEEDED, executableFlow.getStatus()); + Assert.assertEquals(emptyExceptions, executableFlow.getExceptions()); + + List wfmodels = monitor.getCompletedWorkflowModels(false); + logger.info("num wf models " + wfmodels.size()); + WorkflowExecutionModel wfModel = wfmodels.get(0); + List names = wfModel.getWorkflowJobNames(); + Assert.assertNotNull(names); + Assert.assertEquals(1, names.size()); + Assert.assertEquals(rootJobName, names.get(0)); + Assert.assertEquals(WorkflowState.SUCCESSFUL, wfModel.getCompletionState()); + Assert.assertEquals(2, specificJobClassNotifications); + Assert.assertFalse(reason, assertionViolated.get()); + EasyMock.verify(jobManager); + + } + + @Test + public void testLongArgFail() throws Throwable + { + final CountDownLatch completionLatch = new CountDownLatch(1); + monitor.registerJobClassNotification(this, rootJobName); + + PythonJob pythonJob = null; + JobDescriptor descriptor = null; + Props props = null; + + descriptor = EasyMock.createMock(JobDescriptor.class); + + props = new Props(); + props.put("input_date", "20110505"); + props.put(AbstractProcessJob.WORKING_DIR, "."); + props.put("type", "python"); + props.put("script", scriptFileFail); + props.put("output_dir", "/data/report-${input_date}"); + props.put("azkaban.flow.id", wfId); + + EasyMock.expect(descriptor.getId()).andReturn(rootJobName).times(1); + EasyMock.expect(descriptor.getProps()).andReturn(props).times(3); + EasyMock.expect(descriptor.getFullPath()).andReturn(".").times(1); + + EasyMock.replay(descriptor); + + pythonJob = new PythonJob(descriptor); + EasyMock.verify(descriptor); + + final IndividualJobExecutableFlow executableFlow = new IndividualJobExecutableFlow(wfId, rootJobName, jobManager); + EasyMock.expect(jobManager.loadJob(rootJobName, props, true)).andReturn(pythonJob).once(); + + monitor.workflowEvent(null, + System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + wfId); + + monitor.workflowEvent(wfId, + System.currentTimeMillis(), + WorkflowAction.START_WORKFLOW, + WorkflowState.NOP, + rootJobName); + + EasyMock.replay(jobManager); + + Assert.assertEquals(Status.READY, executableFlow.getStatus()); + Assert.assertEquals(emptyExceptions, executableFlow.getExceptions()); + + executableFlow.execute( + props, + new FlowCallback() + { + @Override + public void progressMade() + { + } + + @Override + public void completed(Status status) + { + completionLatch.countDown(); + // Lazy failure is used since this doesn't run in the main thread + if (Status.FAILED != status) { + assertionViolated.set(true); + reason = String.format("In executableFlow Callback: status[%s] != Status.FAILED", status); + } + } + }); + + monitor.workflowEvent(wfId, + System.currentTimeMillis(), + WorkflowAction.END_WORKFLOW, + WorkflowState.FAILED, + rootJobName); + + completionLatch.await(4000, TimeUnit.MILLISECONDS); + Assert.assertEquals(Status.FAILED, executableFlow.getStatus()); + + List wfmodels = monitor.getCompletedWorkflowModels(false); + logger.info("num wf models " + wfmodels.size()); + WorkflowExecutionModel wfModel = wfmodels.get(0); + List names = wfModel.getWorkflowJobNames(); + Assert.assertNotNull(names); + Assert.assertEquals(1, names.size()); + Assert.assertEquals(rootJobName, names.get(0)); + Assert.assertEquals(WorkflowState.FAILED, wfModel.getCompletionState()); + Assert.assertEquals(2, specificJobClassNotifications); + Assert.assertFalse(reason, assertionViolated.get()); + EasyMock.verify(jobManager); + + } + + @Override + public void onGlobalNotify(GlobalNotificationType type, ClassStats statsObject) { + } + + @Override + public void onJobNotify(NativeJobClassStats jobStats) { + logger.info("Received job Class Notification for: " + jobStats.getJobClassName()); + specificJobClassNotifications++; + } + + @Override + public void onWorkflowNotify(NativeWorkflowClassStats wfStats) { + } +} diff --git a/azkaban/src/unit/azkaban/monitor/MonitorImplTest.java b/azkaban/src/unit/azkaban/monitor/MonitorImplTest.java new file mode 100644 index 0000000..30d90c2 --- /dev/null +++ b/azkaban/src/unit/azkaban/monitor/MonitorImplTest.java @@ -0,0 +1,344 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import java.util.Calendar; +import java.util.List; + +import junit.framework.Assert; + +import org.apache.log4j.Logger; +import org.easymock.classextension.EasyMock; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import azkaban.common.utils.Props; +import azkaban.jobcontrol.impl.jobs.ResourceThrottledJob; +import azkaban.jobcontrol.impl.jobs.RetryingJob; +import azkaban.jobs.builtin.ProcessJob; +import azkaban.monitor.MonitorInterface.GlobalNotificationType; +import azkaban.monitor.MonitorInterface.JobState; +import azkaban.monitor.MonitorInterface.WorkflowState; +import azkaban.monitor.MonitorInternalInterface.JobAction; +import azkaban.monitor.MonitorInternalInterface.WorkflowAction; +import azkaban.monitor.stats.ClassStats; +import azkaban.monitor.stats.NativeGlobalStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +/** + * This is a variety of tests for the monitoring system, focusing on retry jobs + * and the notification mechanism. + * + */ +public class MonitorImplTest implements MonitorListener { + private static Logger logger = Logger.getLogger(MonitorImplTest.class); + + private MonitorInterface external; + private MonitorInternalInterface internal; + + private int globalNotifications; + private int anyWorkflowClassNotifications; + private int anyJobClassNotifications; + private int specificWorkflowClassNotifications; + private int specificJobClassNotifications; + + @Before + public void setUp() throws Exception + { + MonitorImpl impl = (MonitorImpl)MonitorImpl.getMonitor(); + external = impl; + internal = impl; + } + + @After + public void tearDown() throws Exception { + MonitorImpl.unsetMonitor(); + } + + @Test + public void testRetries() throws Exception { + String wfId = "25"; + String rootJobName = "MyJob"; + Props props = new Props(); + Props replyProps = new Props(); + props.put("azkaban.flow.id", wfId); + + // This is the inner job for the retry job. + ProcessJob rootJob = EasyMock.createStrictMock(ProcessJob.class); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.replay(rootJob); + + // Create the retrying job; + RetryingJob retryingJob = new RetryingJob(rootJob, 4, 2); + + // Do in order + // Not that if core logic changes, this choreography needs + // updating. + EasyMock.reset(rootJob); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.expect(rootJob.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + rootJob.run(); + EasyMock.expectLastCall().andThrow(new RuntimeException()); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.expect(rootJob.isCanceled()).andReturn(false); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + + EasyMock.expect(rootJob.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + rootJob.run(); + EasyMock.expectLastCall().andThrow(new RuntimeException()); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.expect(rootJob.isCanceled()).andReturn(false); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + + EasyMock.expect(rootJob.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + rootJob.run(); + EasyMock.expect(rootJob.getJobGeneratedProperties()).andReturn(replyProps).anyTimes(); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + + EasyMock.replay(rootJob); + + // We need to fields some events related to a hypothetical workflow and + // initiating the retry job. + MonitorImpl.getInternalMonitorInterface().workflowEvent(null, System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, WorkflowState.NOP, rootJobName); + MonitorImpl.getInternalMonitorInterface().workflowEvent(wfId, System.currentTimeMillis(), + WorkflowAction.START_WORKFLOW, WorkflowState.NOP, rootJobName); + MonitorImpl.getInternalMonitorInterface().jobEvent( + retryingJob, + System.currentTimeMillis(), + JobAction.START_WORKFLOW_JOB, JobState.NOP); + + // Run the retry job + retryingJob.run(); + + // Close out the retry job and workflow with events. + MonitorImpl.getInternalMonitorInterface().jobEvent( + retryingJob, + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, JobState.SUCCESSFUL); + MonitorImpl.getInternalMonitorInterface().workflowEvent(wfId, System.currentTimeMillis(), + WorkflowAction.END_WORKFLOW, WorkflowState.SUCCESSFUL, null); + + // Get the job stats and test a few values. + NativeJobClassStats jobStats = external.getJobClassStatsByName(rootJobName); + Assert.assertNotNull(jobStats); + + Assert.assertEquals(1, jobStats.getNumRetryJobStarts()); + Assert.assertEquals(3, jobStats.getNumJobTries()); + Assert.assertEquals(2, jobStats.getNumTimesJobFailed()); + Assert.assertEquals(1, jobStats.getNumTimesJobSuccessful()); + + logger.info("End testRetries"); + } + + @Test + public void testResourceThrottle() throws Exception { + String wfId = "26"; + String rootJobName = "ResourceJob"; + long delayTime = 3000L; + // Do a second Job + ProcessJob rootJob = EasyMock.createStrictMock(ProcessJob.class); + Props props = new Props(); + Props replyProps = new Props(); + props.put("azkaban.flow.id", wfId); + TimeDelayJobLock tjl = new TimeDelayJobLock(delayTime); + + EasyMock.expect(rootJob.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.replay(rootJob); + + ResourceThrottledJob resourceJob = new ResourceThrottledJob(rootJob, tjl); + + // Do in order + // Not that if core logic changes, this choreography needs + // updating. + EasyMock.reset(rootJob); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.expect(rootJob.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + rootJob.run(); + EasyMock.expect(rootJob.getJobGeneratedProperties()).andReturn(replyProps).anyTimes(); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.replay(rootJob); + + // We need to fields some events related to a hypothetical workflow and + // initiating the retry job. + MonitorImpl.getInternalMonitorInterface().workflowEvent(null, System.currentTimeMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, WorkflowState.NOP, rootJobName); + MonitorImpl.getInternalMonitorInterface().workflowEvent(wfId, System.currentTimeMillis(), + WorkflowAction.START_WORKFLOW, WorkflowState.NOP, rootJobName); + MonitorImpl.getInternalMonitorInterface().jobEvent( + resourceJob, + System.currentTimeMillis(), + JobAction.START_WORKFLOW_JOB, JobState.NOP); + + resourceJob.run(); + + // Close out the retry job and workflow with events. + MonitorImpl.getInternalMonitorInterface().jobEvent( + resourceJob, + System.currentTimeMillis(), + JobAction.END_WORKFLOW_JOB, JobState.SUCCESSFUL); + MonitorImpl.getInternalMonitorInterface().workflowEvent(wfId, System.currentTimeMillis(), + WorkflowAction.END_WORKFLOW, WorkflowState.SUCCESSFUL, null); + + // Get the job stats and test the wait lock value. + NativeJobClassStats jobStats = external.getJobClassStatsByName(rootJobName); + Assert.assertNotNull(jobStats); + // account for non-determinism of thread.sleep(). + long epsilon = 10; + Assert.assertTrue(String.format( + "Resource throttled wait time should be >= %d but was %f", + delayTime, jobStats.getTotalResourceThrottledWaitTime()), + jobStats.getTotalResourceThrottledWaitTime() >= (delayTime - epsilon)); + + + logger.info("End testResourceThrottle"); + } + + @Test + public void testNotifications() throws Exception { + String wfId = "21"; + String rootJobName = "root"; + Props props = new Props(); + props.put("azkaban.flow.id", wfId); + + ProcessJob rootJob = EasyMock.createStrictMock(ProcessJob.class); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.expect(rootJob.getProps()).andReturn(props).anyTimes(); + EasyMock.expect(rootJob.getId()).andReturn(rootJobName).anyTimes(); + EasyMock.replay(rootJob); + + Calendar baseCalendar = Calendar.getInstance(); + baseCalendar.setTimeInMillis(0L); // clear out ms field + baseCalendar.set(2010, 9, 25, 9, 25, 0); + long baseTimeMS = baseCalendar.getTimeInMillis(); + + external.registerGlobalNotification(this, GlobalNotificationType.GLOBAL_STATS_CHANGE); + external.registerGlobalNotification(this, GlobalNotificationType.ANY_WORKFLOW_CLASS_STATS_CHANGE); + external.registerGlobalNotification(this, GlobalNotificationType.ANY_JOB_CLASS_STATS_CHANGE); + external.registerWorkflowClassNotification(this, rootJob.getId()); + external.registerJobClassNotification(this, rootJob.getId()); + + Calendar scheduleTime = Calendar.getInstance(); + scheduleTime.setTimeInMillis(baseTimeMS); + + internal.workflowEvent(null, + scheduleTime.getTimeInMillis(), + WorkflowAction.SCHEDULE_WORKFLOW, + WorkflowState.NOP, + rootJobName); + + Calendar startTime = Calendar.getInstance(); + startTime.setTimeInMillis(baseTimeMS + 1000); + internal.workflowEvent(wfId, + startTime.getTimeInMillis(), + WorkflowAction.START_WORKFLOW, + WorkflowState.NOP, + rootJobName); + + Calendar jobStartTime = Calendar.getInstance(); + jobStartTime.setTimeInMillis(baseTimeMS + 2000); + internal.jobEvent( + rootJob, + jobStartTime.getTimeInMillis(), + JobAction.START_WORKFLOW_JOB, + JobState.NOP); + + Calendar jobEndTime = Calendar.getInstance(); + jobEndTime.setTimeInMillis(baseTimeMS + 3000); + internal.jobEvent( + rootJob, + jobEndTime.getTimeInMillis(), + JobAction.END_WORKFLOW_JOB, + JobState.SUCCESSFUL); + + Calendar endTime = Calendar.getInstance(); + endTime.setTimeInMillis(baseTimeMS + 4000); + internal.workflowEvent(wfId, + endTime.getTimeInMillis(), + WorkflowAction.END_WORKFLOW, + WorkflowState.SUCCESSFUL, + null); + + Assert.assertEquals(5, globalNotifications); + Assert.assertEquals(3, anyWorkflowClassNotifications); + Assert.assertEquals(2, anyJobClassNotifications); + Assert.assertEquals(3, specificWorkflowClassNotifications); + Assert.assertEquals(2, specificJobClassNotifications); + + NativeGlobalStats globalStats = external.getGlobalAzkabanStats(); + Assert.assertEquals(1, globalStats.getTotalWorkflowsStarted()); + Assert.assertEquals(1, globalStats.getTotalJobsStarted()); + + NativeWorkflowClassStats wfClassStats = external.getWorkflowClassStatsByRootJobName(rootJob.getId()); + Assert.assertNotNull(wfClassStats); + Assert.assertEquals(wfClassStats.getAvgWorkflowRunTime(), + (double)(endTime.getTimeInMillis() - startTime.getTimeInMillis())); + + NativeJobClassStats jobClassStats = external.getJobClassStatsByName(rootJob.getId()); + Assert.assertNotNull(jobClassStats); + Assert.assertEquals(jobClassStats.getAvgJobRunTime(), + (double)(jobEndTime.getTimeInMillis() - jobStartTime.getTimeInMillis())); + + external.deregisterNotifications(this); + List notifiers = ((MonitorImpl)external).getAllNotifiers(); + Assert.assertNotNull(notifiers); + Assert.assertEquals(0, notifiers.size()); + + logger.info("End testNotifications"); + } + + @Override + public void onGlobalNotify(GlobalNotificationType type, ClassStats statsObject) { + switch(type) { + case GLOBAL_STATS_CHANGE: + logger.info("Received Global Notification"); + Assert.assertNotNull(statsObject); + Assert.assertTrue(statsObject instanceof NativeGlobalStats); + globalNotifications++; + break; + case ANY_WORKFLOW_CLASS_STATS_CHANGE: + logger.info("Received Any Workflow Class Notification"); + Assert.assertNotNull(statsObject); + Assert.assertTrue(statsObject instanceof NativeWorkflowClassStats); + anyWorkflowClassNotifications++; + break; + case ANY_JOB_CLASS_STATS_CHANGE: + logger.info("Received Any Job Class Notification"); + Assert.assertNotNull(statsObject); + Assert.assertTrue(statsObject instanceof NativeJobClassStats); + anyJobClassNotifications++; + break; + } + } + + @Override + public void onJobNotify(NativeJobClassStats jobStats) { + logger.info("Received job Class Notification for: " + jobStats.getJobClassName()); + specificJobClassNotifications++; + } + + @Override + public void onWorkflowNotify(NativeWorkflowClassStats wfStats) { + logger.info("Received wf Class Notification for: " + wfStats.getWorkflowRootName()); + specificWorkflowClassNotifications++; + } +} diff --git a/azkaban/src/unit/azkaban/monitor/MonitorIntegrationTest.java b/azkaban/src/unit/azkaban/monitor/MonitorIntegrationTest.java new file mode 100644 index 0000000..daa3be7 --- /dev/null +++ b/azkaban/src/unit/azkaban/monitor/MonitorIntegrationTest.java @@ -0,0 +1,109 @@ +package azkaban.monitor; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.concurrent.CountDownLatch; + +import org.apache.commons.io.FileUtils; +import org.joda.time.DateTime; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import azkaban.app.AzkabanApplication; +import azkaban.flow.ExecutableFlow; +import azkaban.flow.FlowManager; +import azkaban.monitor.MonitorInterface.GlobalNotificationType; +import azkaban.monitor.stats.ClassStats; +import azkaban.monitor.stats.NativeGlobalStats; +import azkaban.monitor.stats.NativeJobClassStats; +import azkaban.monitor.stats.NativeWorkflowClassStats; + +public class MonitorIntegrationTest { + + private File testScratchPlace; + private File jobsDir; + private File logDir; + private File tempDir; + private AzkabanApplication application; + private MonitorImpl monitor; + + private File makeScratchDir(String name) throws IOException { + File dir = new File(testScratchPlace, "jobs"); + FileUtils.forceMkdir(dir); + return dir; + } + + @Before + public void setUp() throws IOException { + testScratchPlace = File.createTempFile("scratch", "", new File("dist")); + FileUtils.forceDelete(testScratchPlace); + FileUtils.forceMkdir(testScratchPlace); + System.out.println("Using test scratch place of: " + + testScratchPlace.getAbsolutePath()); + + this.jobsDir = makeScratchDir("jobs"); + this.logDir = makeScratchDir("logs"); + this.tempDir = makeScratchDir("tmp"); + FileUtils.writeLines(new File(jobsDir, "hw.job"), Arrays.asList( + "type=command", "command=echo hello world")); + MonitorImpl.unsetMonitor(); + + + application = new AzkabanApplication(Arrays + .asList(jobsDir), logDir, tempDir, false); + monitor = MonitorImpl.getMonitor(); + + } + + @Test(timeout=10000) + public void testSingleJob() throws IOException, InterruptedException { + // The goal of this test is not to check correctness of the book- + // keeping, but rather to make sure that the book-keeping happens at + // all. + + final CountDownLatch countDown = new CountDownLatch(1); + monitor.registerWorkflowClassNotification(new MonitorListener() { + + @Override + public void onWorkflowNotify(NativeWorkflowClassStats wfStats) { + System.out.println("In onWorkflowNotify!"); + if (wfStats.getNumTimesWorkflowSuccessful() > 0) { + countDown.countDown(); + } + } + + @Override + public void onJobNotify(NativeJobClassStats jobStats) { } + + @Override + public void onGlobalNotify(GlobalNotificationType type, + ClassStats statsObject) { } + }, "hw"); + final FlowManager flowManager = application.getAllFlows(); + ExecutableFlow flowToRun = flowManager.createNewExecutableFlow("hw"); + application.getJobExecutorManager().execute(flowToRun); + + countDown.await(); + // check a bunch of statistics + NativeGlobalStats globalStats = monitor.getGlobalAzkabanStats(); + Assert.assertEquals(1, globalStats.getHighFlowId()); + Assert.assertEquals(1, globalStats.getTotalWorkflowsStarted()); + Assert.assertEquals(1, globalStats.getTotalWorkflowsSuccessful()); + Assert.assertEquals(1, globalStats.getTotalJobsStarted()); + Assert.assertEquals(1, globalStats.getTotalJobsSuccessful()); + + Assert.assertEquals("The number of workflows is incorrect.", 1, monitor + .getNumberOfWorkflows()); + } + + @Test + public void testScheduledJob() { + DateTime scheduleTime = new DateTime().plusDays(1); + application.getScheduleManager().schedule("hw", scheduleTime, false); + + Assert.assertEquals(1, monitor.getGlobalAzkabanStats().getTotalWorkflowsScheduled()); + } +} diff --git a/azkaban/src/unit/azkaban/monitor/TimeDelayJobLock.java b/azkaban/src/unit/azkaban/monitor/TimeDelayJobLock.java new file mode 100644 index 0000000..a8430b5 --- /dev/null +++ b/azkaban/src/unit/azkaban/monitor/TimeDelayJobLock.java @@ -0,0 +1,71 @@ +/* + * Copyright 2010 Adconion, Inc + * + * 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 azkaban.monitor; + +import java.util.concurrent.TimeUnit; + +import azkaban.jobcontrol.impl.jobs.locks.JobLock; + +/** + * Test job lock that enforces a simple delay while acquiring it. + */ +public class TimeDelayJobLock implements JobLock { + private long waitTimeMs; + + /** + * constructor + * @param waitTimeMs + * time in ms to simulate waiting on resources. + */ + public TimeDelayJobLock(long waitTimeMs) { + this.waitTimeMs = waitTimeMs; + } + + @Override + public void acquireLock() throws InterruptedException { + Thread.sleep(waitTimeMs); + } + + @Override + public long getLockAcquireTime(TimeUnit unit) { + // TODO Auto-generated method stub + return 0; + } + + @Override + public long getLockHeldTime(TimeUnit unit) { + // TODO Auto-generated method stub + return 0; + } + + @Override + public Status getStatus() { + // TODO Auto-generated method stub + return null; + } + + @Override + public long getTotalLockTime(TimeUnit unit) { + // TODO Auto-generated method stub + return 0; + } + + @Override + public void releaseLock() { + // TODO Auto-generated method stub + + } + +} diff --git a/lib/commons-math-2.1.jar b/lib/commons-math-2.1.jar new file mode 100644 index 0000000..43b4b36 Binary files /dev/null and b/lib/commons-math-2.1.jar differ