001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *      http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.oozie;
020
021import java.io.IOException;
022import java.io.StringReader;
023import java.io.Writer;
024import java.util.ArrayList;
025import java.util.Date;
026import java.util.HashMap;
027import java.util.HashSet;
028import java.util.List;
029import java.util.Map;
030import java.util.Properties;
031import java.util.Set;
032import java.util.StringTokenizer;
033
034import org.apache.hadoop.conf.Configuration;
035import org.apache.oozie.client.CoordinatorJob;
036import org.apache.oozie.client.OozieClient;
037import org.apache.oozie.client.WorkflowJob;
038import org.apache.oozie.command.CommandException;
039import org.apache.oozie.command.OperationType;
040import org.apache.oozie.command.wf.BulkWorkflowXCommand;
041import org.apache.oozie.command.wf.CompletedActionXCommand;
042import org.apache.oozie.command.wf.DefinitionXCommand;
043import org.apache.oozie.command.wf.ExternalIdXCommand;
044import org.apache.oozie.command.wf.JobXCommand;
045import org.apache.oozie.command.wf.JobsXCommand;
046import org.apache.oozie.command.wf.KillXCommand;
047import org.apache.oozie.command.wf.ReRunXCommand;
048import org.apache.oozie.command.wf.ResumeXCommand;
049import org.apache.oozie.command.wf.StartXCommand;
050import org.apache.oozie.command.wf.SubmitHiveXCommand;
051import org.apache.oozie.command.wf.SubmitHttpXCommand;
052import org.apache.oozie.command.wf.SubmitMRXCommand;
053import org.apache.oozie.command.wf.SubmitPigXCommand;
054import org.apache.oozie.command.wf.SubmitSqoopXCommand;
055import org.apache.oozie.command.wf.SubmitXCommand;
056import org.apache.oozie.command.wf.SuspendXCommand;
057import org.apache.oozie.command.wf.WorkflowActionInfoXCommand;
058import org.apache.oozie.executor.jpa.JPAExecutorException;
059import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
060import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
061import org.apache.oozie.service.CallableQueueService;
062import org.apache.oozie.service.DagXLogInfoService;
063import org.apache.oozie.service.Services;
064import org.apache.oozie.service.XLogService;
065import org.apache.oozie.util.ParamChecker;
066import org.apache.oozie.util.XCallable;
067import org.apache.oozie.util.XConfiguration;
068import org.apache.oozie.util.XLog;
069import org.apache.oozie.util.XLogAuditFilter;
070import org.apache.oozie.util.XLogFilter;
071import org.apache.oozie.util.XLogUserFilterParam;
072
073/**
074 * The DagEngine provides all the DAG engine functionality for WS calls.
075 */
076public class DagEngine extends BaseEngine {
077
078    private static final int HIGH_PRIORITY = 2;
079    private static XLog LOG = XLog.getLog(DagEngine.class);
080
081    /**
082     * Create a system Dag engine, with no user and no group.
083     */
084    public DagEngine() {
085
086    }
087
088    /**
089     * Create a Dag engine to perform operations on behave of a user.
090     *
091     * @param user user name.
092     */
093    public DagEngine(String user) {
094        this();
095
096        this.user = ParamChecker.notEmpty(user, "user");
097    }
098
099    /**
100     * Submit a workflow job. <p> It validates configuration properties.
101     *
102     * @param conf job configuration.
103     * @param startJob indicates if the job should be started or not.
104     * @return the job Id.
105     * @throws DagEngineException thrown if the job could not be created.
106     */
107    @Override
108    public String submitJob(Configuration conf, boolean startJob) throws DagEngineException {
109        validateSubmitConfiguration(conf);
110
111        try {
112            String jobId;
113            SubmitXCommand submit = new SubmitXCommand(conf);
114            jobId = submit.call();
115            if (startJob) {
116                start(jobId);
117            }
118            return jobId;
119        }
120        catch (CommandException ex) {
121            throw new DagEngineException(ex);
122        }
123    }
124
125    /**
126     * Submit a workflow through a coordinator. It validates configuration properties.
127     * @param conf job conf
128     * @param parentId parent of workflow
129     * @return
130     * @throws DagEngineException
131     */
132    public String submitJobFromCoordinator(Configuration conf, String parentId) throws DagEngineException {
133        validateSubmitConfiguration(conf);
134        try {
135            String jobId;
136            SubmitXCommand submit = new SubmitXCommand(conf, parentId);
137            jobId = submit.call();
138            start(jobId);
139            return jobId;
140        }
141        catch (CommandException ex) {
142            throw new DagEngineException(ex);
143        }
144    }
145
146    /**
147     * Submit a pig/hive/mapreduce job through HTTP.
148     * <p>
149     * It validates configuration properties.
150     *
151     * @param conf job configuration.
152     * @param jobType job type - can be "pig", "hive", "sqoop" or "mapreduce".
153     * @return the job Id.
154     * @throws DagEngineException thrown if the job could not be created.
155     */
156    public String submitHttpJob(Configuration conf, String jobType) throws DagEngineException {
157        validateSubmitConfiguration(conf);
158
159        try {
160            String jobId;
161            SubmitHttpXCommand submit = null;
162            if (jobType.equals("pig")) {
163                submit = new SubmitPigXCommand(conf);
164            }
165            else if (jobType.equals("mapreduce")) {
166                submit = new SubmitMRXCommand(conf);
167            }
168            else if (jobType.equals("hive")) {
169                submit = new SubmitHiveXCommand(conf);
170            }
171            else if (jobType.equals("sqoop")) {
172                submit = new SubmitSqoopXCommand(conf);
173            }
174
175            jobId = submit.call();
176            start(jobId);
177            return jobId;
178        }
179        catch (CommandException ex) {
180            throw new DagEngineException(ex);
181        }
182    }
183
184    private void validateSubmitConfiguration(Configuration conf) throws DagEngineException {
185        if (conf.get(OozieClient.APP_PATH) == null) {
186            throw new DagEngineException(ErrorCode.E0401, OozieClient.APP_PATH);
187        }
188    }
189
190    /**
191     * Start a job.
192     *
193     * @param jobId job Id.
194     * @throws DagEngineException thrown if the job could not be started.
195     */
196    @Override
197    public void start(String jobId) throws DagEngineException {
198        // Changing to synchronous call from asynchronous queuing to prevent the
199        // loss of command if the queue is full or the queue is lost in case of
200        // failure.
201        try {
202                new StartXCommand(jobId).call();
203        }
204        catch (CommandException e) {
205            throw new DagEngineException(e);
206        }
207    }
208
209    /**
210     * Resume a job.
211     *
212     * @param jobId job Id.
213     * @throws DagEngineException thrown if the job could not be resumed.
214     */
215    @Override
216    public void resume(String jobId) throws DagEngineException {
217        // Changing to synchronous call from asynchronous queuing to prevent the
218        // loss of command if the queue is full or the queue is lost in case of
219        // failure.
220        try {
221                new ResumeXCommand(jobId).call();
222        }
223        catch (CommandException e) {
224            throw new DagEngineException(e);
225        }
226    }
227
228    /**
229     * Suspend a job.
230     *
231     * @param jobId job Id.
232     * @throws DagEngineException thrown if the job could not be suspended.
233     */
234    @Override
235    public void suspend(String jobId) throws DagEngineException {
236        // Changing to synchronous call from asynchronous queuing to prevent the
237        // loss of command if the queue is full or the queue is lost in case of
238        // failure.
239        try {
240                        new SuspendXCommand(jobId).call();
241        }
242        catch (CommandException e) {
243            throw new DagEngineException(e);
244        }
245    }
246
247    /**
248     * Kill a job.
249     *
250     * @param jobId job Id.
251     * @throws DagEngineException thrown if the job could not be killed.
252     */
253    @Override
254    public void kill(String jobId) throws DagEngineException {
255        // Changing to synchronous call from asynchronous queuing to prevent the
256        // loss of command if the queue is full or the queue is lost in case of
257        // failure.
258        try {
259                        new KillXCommand(jobId).call();
260                        LOG.info("User " + user + " killed the WF job " + jobId);
261        }
262        catch (CommandException e) {
263            throw new DagEngineException(e);
264        }
265    }
266
267    /* (non-Javadoc)
268     * @see org.apache.oozie.BaseEngine#change(java.lang.String, java.lang.String)
269     */
270    @Override
271    public void change(String jobId, String changeValue) throws DagEngineException {
272        // This code should not be reached.
273        throw new DagEngineException(ErrorCode.E1017);
274    }
275
276    /**
277     * Rerun a job.
278     *
279     * @param jobId job Id to rerun.
280     * @param conf configuration information for the rerun.
281     * @throws DagEngineException thrown if the job could not be rerun.
282     */
283    @Override
284    public void reRun(String jobId, Configuration conf) throws DagEngineException {
285        try {
286            WorkflowJobBean wfBean = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW, jobId);
287            Configuration wfConf = new XConfiguration(new StringReader(wfBean.getConf()));
288            XConfiguration.copy(conf, wfConf);
289            validateReRunConfiguration(wfConf);
290            new ReRunXCommand(jobId, wfConf).call();
291        }
292        catch (CommandException ex) {
293            throw new DagEngineException(ex);
294        }
295        catch (JPAExecutorException ex) {
296            throw new DagEngineException(ex);
297        }
298        catch (IOException ex) {
299            throw new DagEngineException(ErrorCode.E0803, ex.getMessage());
300        }
301    }
302
303    private void validateReRunConfiguration(Configuration conf) throws DagEngineException {
304        if (conf.get(OozieClient.APP_PATH) == null) {
305            throw new DagEngineException(ErrorCode.E0401, OozieClient.APP_PATH);
306        }
307        if (conf.get(OozieClient.RERUN_SKIP_NODES) == null && conf.get(OozieClient.RERUN_FAIL_NODES) == null) {
308            throw new DagEngineException(ErrorCode.E0401, OozieClient.RERUN_SKIP_NODES + " OR "
309                    + OozieClient.RERUN_FAIL_NODES);
310        }
311        if (conf.get(OozieClient.RERUN_SKIP_NODES) != null && conf.get(OozieClient.RERUN_FAIL_NODES) != null) {
312            throw new DagEngineException(ErrorCode.E0404, OozieClient.RERUN_SKIP_NODES + " OR "
313                    + OozieClient.RERUN_FAIL_NODES);
314        }
315    }
316
317    /**
318     * Process an action callback.
319     *
320     * @param actionId the action Id.
321     * @param externalStatus the action external status.
322     * @param actionData the action output data, <code>null</code> if none.
323     * @throws DagEngineException thrown if the callback could not be processed.
324     */
325    public void processCallback(String actionId, String externalStatus, Properties actionData)
326            throws DagEngineException {
327        XLog.Info.get().clearParameter(XLogService.GROUP);
328        XLog.Info.get().clearParameter(XLogService.USER);
329        XCallable<Void> command = null;
330
331                command = new CompletedActionXCommand(actionId, externalStatus,
332                                actionData, HIGH_PRIORITY);
333        if (!Services.get().get(CallableQueueService.class).queue(command)) {
334            LOG.warn(XLog.OPS, "queue is full or system is in SAFEMODE, ignoring callback");
335        }
336    }
337
338    /**
339     * Return the info about a job.
340     *
341     * @param jobId job Id.
342     * @return the workflow job info.
343     * @throws DagEngineException thrown if the job info could not be obtained.
344     */
345    @Override
346    public WorkflowJob getJob(String jobId) throws DagEngineException {
347        try {
348                return new JobXCommand(jobId).call();
349        }
350        catch (CommandException ex) {
351            throw new DagEngineException(ex);
352        }
353    }
354
355    /**
356     * Return the info about a job with actions subset.
357     *
358     * @param jobId job Id
359     * @param start starting from this index in the list of actions belonging to the job
360     * @param length number of actions to be returned
361     * @return the workflow job info.
362     * @throws DagEngineException thrown if the job info could not be obtained.
363     */
364    @Override
365    public WorkflowJob getJob(String jobId, int start, int length) throws DagEngineException {
366        try {
367                        return new JobXCommand(jobId, start, length).call();
368        }
369        catch (CommandException ex) {
370            throw new DagEngineException(ex);
371        }
372    }
373
374    /**
375     * Return the a job definition.
376     *
377     * @param jobId job Id.
378     * @return the job definition.
379     * @throws DagEngineException thrown if the job definition could no be obtained.
380     */
381    @Override
382    public String getDefinition(String jobId) throws DagEngineException {
383        try {
384            return new DefinitionXCommand(jobId).call();
385        }
386        catch (CommandException ex) {
387            throw new DagEngineException(ex);
388        }
389    }
390
391    /**
392     * Stream the log of a job.
393     *
394     * @param jobId job Id.
395     * @param writer writer to stream the log to.
396     * @param params additional parameters from the request
397     * @throws IOException thrown if the log cannot be streamed.
398     * @throws DagEngineException thrown if there is error in getting the Workflow Information for jobId.
399     */
400    @Override
401    public void streamLog(String jobId, Writer writer, Map<String, String[]> params) throws IOException,
402            DagEngineException {
403        streamJobLog(jobId, writer, params, LOG_TYPE.LOG);
404    }
405
406    /**
407     * Stream the error log of a job.
408     *
409     * @param jobId job Id.
410     * @param writer writer to stream the log to.
411     * @param params additional parameters from the request
412     * @throws IOException thrown if the log cannot be streamed.
413     * @throws DagEngineException thrown if there is error in getting the Workflow Information for jobId.
414     */
415    @Override
416    public void streamErrorLog(String jobId, Writer writer, Map<String, String[]> params) throws IOException,
417            DagEngineException {
418        streamJobLog(jobId, writer, params, LOG_TYPE.ERROR_LOG);
419    }
420
421    /**
422     * Stream the audit log of a job.
423     *
424     * @param jobId job Id.
425     * @param writer writer to stream the log to.
426     * @param params additional parameters from the request
427     * @throws IOException thrown if the log cannot be streamed.
428     * @throws DagEngineException thrown if there is error in getting the Workflow Information for jobId.
429     */
430    @Override
431    public void streamAuditLog(String jobId, Writer writer, Map<String, String[]> params) throws IOException,
432            DagEngineException {
433        try {
434            streamJobLog(new XLogAuditFilter(new XLogUserFilterParam(params)),jobId, writer, params, LOG_TYPE.AUDIT_LOG);
435        }
436        catch (CommandException e) {
437            throw new IOException(e);
438        }
439    }
440
441    private void streamJobLog(String jobId, Writer writer, Map<String, String[]> params, LOG_TYPE logType)
442            throws IOException, DagEngineException {
443        try {
444            streamJobLog(new XLogFilter(new XLogUserFilterParam(params)), jobId, writer, params, logType);
445        }
446        catch (Exception e) {
447            throw new IOException(e);
448        }
449    }
450
451    private void streamJobLog(XLogFilter filter, String jobId, Writer writer, Map<String, String[]> params, LOG_TYPE logType)
452            throws IOException, DagEngineException {
453        try {
454            filter.setParameter(DagXLogInfoService.JOB, jobId);
455            WorkflowJob job = getJob(jobId);
456            Date lastTime = job.getEndTime();
457            if (lastTime == null) {
458                lastTime = job.getLastModifiedTime();
459            }
460            fetchLog(filter, job.getCreatedTime(), lastTime, writer, params, logType);
461        }
462        catch (Exception e) {
463            throw new IOException(e);
464        }
465    }
466
467    private static final Set<String> FILTER_NAMES = new HashSet<String>();
468
469    static {
470        FILTER_NAMES.add(OozieClient.FILTER_USER);
471        FILTER_NAMES.add(OozieClient.FILTER_NAME);
472        FILTER_NAMES.add(OozieClient.FILTER_GROUP);
473        FILTER_NAMES.add(OozieClient.FILTER_STATUS);
474        FILTER_NAMES.add(OozieClient.FILTER_ID);
475        FILTER_NAMES.add(OozieClient.FILTER_CREATED_TIME_START);
476        FILTER_NAMES.add(OozieClient.FILTER_CREATED_TIME_END);
477        FILTER_NAMES.add(OozieClient.FILTER_SORT_BY);
478    }
479
480    /**
481     * Validate a jobs filter.
482     *
483     * @param filter filter to validate.
484     * @return the parsed filter.
485     * @throws DagEngineException thrown if the filter is invalid.
486     */
487    protected Map<String, List<String>> parseFilter(String filter) throws DagEngineException {
488        Map<String, List<String>> map = new HashMap<String, List<String>>();
489        if (filter != null) {
490            StringTokenizer st = new StringTokenizer(filter, ";");
491            while (st.hasMoreTokens()) {
492                String token = st.nextToken();
493                if (token.contains("=")) {
494                    String[] pair = token.split("=");
495                    if (pair.length != 2) {
496                        throw new DagEngineException(ErrorCode.E0420, filter,
497                                "elements must be semicolon-separated name=value pairs");
498                    }
499                    pair[0] = pair[0].toLowerCase();
500                    if (!FILTER_NAMES.contains(pair[0])) {
501                        throw new DagEngineException(ErrorCode.E0420, filter, XLog
502                                .format("invalid name [{0}]", pair[0]));
503                    }
504                    if (pair[0].equals("status")) {
505                        try {
506                            WorkflowJob.Status.valueOf(pair[1]);
507                        }
508                        catch (IllegalArgumentException ex) {
509                            throw new DagEngineException(ErrorCode.E0420, filter, XLog.format("invalid status [{0}]",
510                                                                                              pair[1]));
511                        }
512                    }
513                    List<String> list = map.get(pair[0]);
514                    if (list == null) {
515                        list = new ArrayList<String>();
516                        map.put(pair[0], list);
517                    }
518                    list.add(pair[1]);
519                }
520                else {
521                    throw new DagEngineException(ErrorCode.E0420, filter,
522                            "elements must be semicolon-separated name=value pairs");
523                }
524            }
525        }
526        return map;
527    }
528
529    /**
530     * Return the info about a set of jobs.
531     *
532     * @param filter job filter. Refer to the {@link org.apache.oozie.client.OozieClient} for the filter syntax.
533     * @param start offset, base 1.
534     * @param len number of jobs to return.
535     * @return job info for all matching jobs, the jobs don't contain node action information.
536     * @throws DagEngineException thrown if the jobs info could not be obtained.
537     */
538    public WorkflowsInfo getJobs(String filter, int start, int len) throws DagEngineException {
539        Map<String, List<String>> filterList = parseFilter(filter);
540        try {
541                        return new JobsXCommand(filterList, start, len).call();
542        }
543        catch (CommandException dce) {
544            throw new DagEngineException(dce);
545        }
546    }
547
548    /**
549     * Return the workflow Job ID for an external ID. <p> This is reverse lookup for recovery purposes.
550     *
551     * @param externalId external ID provided at job submission time.
552     * @return the associated workflow job ID if any, <code>null</code> if none.
553     * @throws DagEngineException thrown if the lookup could not be done.
554     */
555    @Override
556    public String getJobIdForExternalId(String externalId) throws DagEngineException {
557        try {
558                        return new ExternalIdXCommand(externalId).call();
559        }
560        catch (CommandException dce) {
561            throw new DagEngineException(dce);
562        }
563    }
564
565    @Override
566    public CoordinatorJob getCoordJob(String jobId) throws BaseEngineException {
567        throw new BaseEngineException(new XException(ErrorCode.E0301, "cannot get a coordinator job from DagEngine"));
568    }
569
570    @Override
571    public CoordinatorJob getCoordJob(String jobId, String filter, int start, int length, boolean desc)
572            throws BaseEngineException {
573        throw new BaseEngineException(new XException(ErrorCode.E0301, "cannot get a coordinator job from DagEngine"));
574    }
575
576    public WorkflowActionBean getWorkflowAction(String actionId) throws BaseEngineException {
577        try {
578                        return new WorkflowActionInfoXCommand(actionId).call();
579        }
580        catch (CommandException ex) {
581            throw new BaseEngineException(ex);
582        }
583    }
584
585    /* (non-Javadoc)
586     * @see org.apache.oozie.BaseEngine#dryRunSubmit(org.apache.hadoop.conf.Configuration)
587     */
588    @Override
589    public String dryRunSubmit(Configuration conf) throws BaseEngineException {
590        try {
591            SubmitXCommand submit = new SubmitXCommand(true, conf);
592            return submit.call();
593        } catch (CommandException ex) {
594            throw new DagEngineException(ex);
595        }
596    }
597
598    /**
599     * Return the status for a Job ID
600     *
601     * @param jobId job Id.
602     * @return the job's status
603     * @throws DagEngineException thrown if the job's status could not be obtained
604     */
605    @Override
606    public String getJobStatus(String jobId) throws DagEngineException {
607        try {
608            WorkflowJobBean wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_STATUS, jobId);
609            return wfJob.getStatusStr();
610        }
611        catch (JPAExecutorException ex) {
612            throw new DagEngineException(ex);
613        }
614    }
615
616    @Override
617    public void enableSLAAlert(String id, String actions, String dates, String childIds) throws BaseEngineException {
618        throw new BaseEngineException(new XException(ErrorCode.E0301, "Not supported for workflow"));
619    }
620
621    @Override
622    public void disableSLAAlert(String id, String actions, String dates, String childIds) throws BaseEngineException {
623        throw new BaseEngineException(new XException(ErrorCode.E0301, "Not supported for workflow"));
624    }
625
626    @Override
627    public void changeSLA(String id, String actions, String dates, String childIds, String newParams) throws BaseEngineException {
628        throw new BaseEngineException(new XException(ErrorCode.E0301, "Not supported for workflow"));
629    }
630
631    /**
632     * return the jobs that've been killed
633     * @param filter Jobs that satisfy the filter will be killed
634     * @param start start index in the database of jobs
635     * @param len maximum number of jobs that will be killed
636     * @return
637     * @throws DagEngineException
638     */
639    public WorkflowsInfo killJobs(String filter, int start, int len) throws DagEngineException {
640        try {
641            Map<String, List<String>> filterList = parseFilter(filter);
642            WorkflowsInfo workflowsInfo = new BulkWorkflowXCommand(filterList, start, len, OperationType.Kill).call();
643            if (workflowsInfo == null) {
644                return new WorkflowsInfo(new ArrayList<WorkflowJobBean>(), 0, 0, 0);
645            }
646            return workflowsInfo;
647        }
648        catch (CommandException ex) {
649            throw new DagEngineException(ex);
650        }
651    }
652
653    /**
654     * return the jobs that've been suspended
655     * @param filter Filter for jobs that will be suspended, can be name, user, group, status, id or combination of any
656     * @param start Offset for the jobs that will be suspended
657     * @param len maximum number of jobs that will be suspended
658     * @return
659     * @throws DagEngineException
660     */
661    public WorkflowsInfo suspendJobs(String filter, int start, int len) throws DagEngineException {
662        try {
663            Map<String, List<String>> filterList = parseFilter(filter);
664            WorkflowsInfo workflowsInfo = new BulkWorkflowXCommand(filterList, start, len, OperationType.Suspend).call();
665            if (workflowsInfo == null) {
666                return new WorkflowsInfo(new ArrayList<WorkflowJobBean>(), 0, 0, 0);
667            }
668            return workflowsInfo;
669        }
670        catch (CommandException ex) {
671            throw new DagEngineException(ex);
672        }
673    }
674
675    /**
676     * return the jobs that've been resumed
677     * @param filter Filter for jobs that will be resumed, can be name, user, group, status, id or combination of any
678     * @param start Offset for the jobs that will be resumed
679     * @param len maximum number of jobs that will be resumed
680     * @return
681     * @throws DagEngineException
682     */
683    public WorkflowsInfo resumeJobs(String filter, int start, int len) throws DagEngineException {
684        try {
685            Map<String, List<String>> filterList = parseFilter(filter);
686            WorkflowsInfo workflowsInfo = new BulkWorkflowXCommand(filterList, start, len, OperationType.Resume).call();
687            if (workflowsInfo == null) {
688                return new WorkflowsInfo(new ArrayList<WorkflowJobBean>(), 0, 0, 0);
689            }
690            return workflowsInfo;
691        }
692        catch (CommandException ex) {
693            throw new DagEngineException(ex);
694        }
695    }
696}