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.command.wf.WorkflowActionRetryInfoXCommand;
059import org.apache.oozie.executor.jpa.JPAExecutorException;
060import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
061import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
062import org.apache.oozie.service.CallableQueueService;
063import org.apache.oozie.service.DagXLogInfoService;
064import org.apache.oozie.service.Services;
065import org.apache.oozie.service.XLogService;
066import org.apache.oozie.service.XLogStreamingService;
067import org.apache.oozie.util.ParamChecker;
068import org.apache.oozie.util.XCallable;
069import org.apache.oozie.util.XConfiguration;
070import org.apache.oozie.util.XLog;
071import org.apache.oozie.util.XLogStreamer;
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 jobId returns jobId of the launched workflow
130     * @throws DagEngineException if the job can't be submitted
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    @Override
392    protected void streamJobLog(XLogStreamer logStreamer, String jobId, Writer writer)
393            throws IOException, DagEngineException {
394        logStreamer.getXLogFilter().setParameter(DagXLogInfoService.JOB, jobId);
395        WorkflowJob job = getJob(jobId);
396        Date lastTime = job.getEndTime();
397        if (lastTime == null) {
398            lastTime = job.getLastModifiedTime();
399        }
400        Services.get().get(XLogStreamingService.class).streamLog(logStreamer, job.getCreatedTime(), lastTime, writer);
401    }
402
403    private static final Set<String> FILTER_NAMES = new HashSet<String>();
404
405    static {
406        FILTER_NAMES.add(OozieClient.FILTER_TEXT);
407        FILTER_NAMES.add(OozieClient.FILTER_USER);
408        FILTER_NAMES.add(OozieClient.FILTER_NAME);
409        FILTER_NAMES.add(OozieClient.FILTER_GROUP);
410        FILTER_NAMES.add(OozieClient.FILTER_STATUS);
411        FILTER_NAMES.add(OozieClient.FILTER_ID);
412        FILTER_NAMES.add(OozieClient.FILTER_CREATED_TIME_START);
413        FILTER_NAMES.add(OozieClient.FILTER_CREATED_TIME_END);
414        FILTER_NAMES.add(OozieClient.FILTER_SORT_BY);
415    }
416
417    /**
418     * Validate a jobs filter.
419     *
420     * @param filter filter to validate.
421     * @return the parsed filter.
422     * @throws DagEngineException thrown if the filter is invalid.
423     */
424    protected Map<String, List<String>> parseFilter(String filter) throws DagEngineException {
425        Map<String, List<String>> map = new HashMap<String, List<String>>();
426        if (filter != null) {
427            StringTokenizer st = new StringTokenizer(filter, ";");
428            while (st.hasMoreTokens()) {
429                String token = st.nextToken();
430                if (token.contains("=")) {
431                    String[] pair = token.split("=");
432                    if (pair.length != 2) {
433                        throw new DagEngineException(ErrorCode.E0420, filter,
434                                "elements must be semicolon-separated name=value pairs");
435                    }
436                    pair[0] = pair[0].toLowerCase();
437                    if (!FILTER_NAMES.contains(pair[0])) {
438                        throw new DagEngineException(ErrorCode.E0420, filter, XLog
439                                .format("invalid name [{0}]", pair[0]));
440                    }
441                    if (pair[0].equals("status")) {
442                        try {
443                            WorkflowJob.Status.valueOf(pair[1]);
444                        }
445                        catch (IllegalArgumentException ex) {
446                            throw new DagEngineException(ErrorCode.E0420, filter, XLog.format("invalid status [{0}]",
447                                                                                              pair[1]));
448                        }
449                    }
450                    List<String> list = map.get(pair[0]);
451                    if (list == null) {
452                        list = new ArrayList<String>();
453                        map.put(pair[0], list);
454                    }
455                    list.add(pair[1]);
456                }
457                else {
458                    throw new DagEngineException(ErrorCode.E0420, filter,
459                            "elements must be semicolon-separated name=value pairs");
460                }
461            }
462        }
463        return map;
464    }
465
466    /**
467     * Return the info about a set of jobs.
468     *
469     * @param filter job filter. Refer to the {@link org.apache.oozie.client.OozieClient} for the filter syntax.
470     * @param start offset, base 1.
471     * @param len number of jobs to return.
472     * @return job info for all matching jobs, the jobs don't contain node action information.
473     * @throws DagEngineException thrown if the jobs info could not be obtained.
474     */
475    public WorkflowsInfo getJobs(String filter, int start, int len) throws DagEngineException {
476        Map<String, List<String>> filterList = parseFilter(filter);
477        try {
478                        return new JobsXCommand(filterList, start, len).call();
479        }
480        catch (CommandException dce) {
481            throw new DagEngineException(dce);
482        }
483    }
484
485    /**
486     * Return the workflow Job ID for an external ID. <p> This is reverse lookup for recovery purposes.
487     *
488     * @param externalId external ID provided at job submission time.
489     * @return the associated workflow job ID if any, <code>null</code> if none.
490     * @throws DagEngineException thrown if the lookup could not be done.
491     */
492    @Override
493    public String getJobIdForExternalId(String externalId) throws DagEngineException {
494        try {
495                        return new ExternalIdXCommand(externalId).call();
496        }
497        catch (CommandException dce) {
498            throw new DagEngineException(dce);
499        }
500    }
501
502    @Override
503    public CoordinatorJob getCoordJob(String jobId) throws BaseEngineException {
504        throw new BaseEngineException(new XException(ErrorCode.E0301, "cannot get a coordinator job from DagEngine"));
505    }
506
507    @Override
508    public CoordinatorJob getCoordJob(String jobId, String filter, int start, int length, boolean desc)
509            throws BaseEngineException {
510        throw new BaseEngineException(new XException(ErrorCode.E0301, "cannot get a coordinator job from DagEngine"));
511    }
512
513    public WorkflowActionBean getWorkflowAction(String actionId) throws BaseEngineException {
514        try {
515                        return new WorkflowActionInfoXCommand(actionId).call();
516        }
517        catch (CommandException ex) {
518            throw new BaseEngineException(ex);
519        }
520    }
521
522    /**
523     * Gets the workflow action retries.
524     *
525     * @param actionId the action id
526     * @return the workflow action retries
527     * @throws BaseEngineException the base engine exception
528     */
529    public List<Map<String, String>> getWorkflowActionRetries(String actionId) throws BaseEngineException {
530        try {
531            return new WorkflowActionRetryInfoXCommand(actionId).call();
532        }
533        catch (CommandException ex) {
534            throw new BaseEngineException(ex);
535        }
536    }
537
538    /* (non-Javadoc)
539     * @see org.apache.oozie.BaseEngine#dryRunSubmit(org.apache.hadoop.conf.Configuration)
540     */
541    @Override
542    public String dryRunSubmit(Configuration conf) throws BaseEngineException {
543        try {
544            SubmitXCommand submit = new SubmitXCommand(true, conf);
545            return submit.call();
546        } catch (CommandException ex) {
547            throw new DagEngineException(ex);
548        }
549    }
550
551    /**
552     * Return the status for a Job ID
553     *
554     * @param jobId job Id.
555     * @return the job's status
556     * @throws DagEngineException thrown if the job's status could not be obtained
557     */
558    @Override
559    public String getJobStatus(String jobId) throws DagEngineException {
560        try {
561            WorkflowJobBean wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_STATUS, jobId);
562            return wfJob.getStatusStr();
563        }
564        catch (JPAExecutorException ex) {
565            throw new DagEngineException(ex);
566        }
567    }
568
569    @Override
570    public void enableSLAAlert(String id, String actions, String dates, String childIds) throws BaseEngineException {
571        throw new BaseEngineException(new XException(ErrorCode.E0301, "Not supported for workflow"));
572    }
573
574    @Override
575    public void disableSLAAlert(String id, String actions, String dates, String childIds) throws BaseEngineException {
576        throw new BaseEngineException(new XException(ErrorCode.E0301, "Not supported for workflow"));
577    }
578
579    @Override
580    public void changeSLA(String id, String actions, String dates, String childIds, String newParams) throws BaseEngineException {
581        throw new BaseEngineException(new XException(ErrorCode.E0301, "Not supported for workflow"));
582    }
583
584    /**
585     * return the jobs that've been killed
586     * @param filter Jobs that satisfy the filter will be killed
587     * @param start start index in the database of jobs
588     * @param len maximum number of jobs that will be killed
589     * @return workflowsInfo return the jobs that've been killed
590     * @throws DagEngineException if the jobs could not be killed
591     */
592    public WorkflowsInfo killJobs(String filter, int start, int len) throws DagEngineException {
593        try {
594            Map<String, List<String>> filterList = parseFilter(filter);
595            WorkflowsInfo workflowsInfo = new BulkWorkflowXCommand(filterList, start, len, OperationType.Kill).call();
596            if (workflowsInfo == null) {
597                return new WorkflowsInfo(new ArrayList<WorkflowJobBean>(), 0, 0, 0);
598            }
599            return workflowsInfo;
600        }
601        catch (CommandException ex) {
602            throw new DagEngineException(ex);
603        }
604    }
605
606    /**
607     * return the jobs that've been suspended
608     * @param filter Filter for jobs that will be suspended, can be name, user, group, status, id or combination of any
609     * @param start Offset for the jobs that will be suspended
610     * @param len maximum number of jobs that will be suspended
611     * @return workflowsInfo return the jobs that've been suspended
612     * @throws DagEngineException if the jobs cloud not be suspended
613     */
614    public WorkflowsInfo suspendJobs(String filter, int start, int len) throws DagEngineException {
615        try {
616            Map<String, List<String>> filterList = parseFilter(filter);
617            WorkflowsInfo workflowsInfo = new BulkWorkflowXCommand(filterList, start, len, OperationType.Suspend).call();
618            if (workflowsInfo == null) {
619                return new WorkflowsInfo(new ArrayList<WorkflowJobBean>(), 0, 0, 0);
620            }
621            return workflowsInfo;
622        }
623        catch (CommandException ex) {
624            throw new DagEngineException(ex);
625        }
626    }
627
628    /**
629     * return the jobs that've been resumed
630     * @param filter Filter for jobs that will be resumed, can be name, user, group, status, id or combination of any
631     * @param start Offset for the jobs that will be resumed
632     * @param len maximum number of jobs that will be resumed
633     * @return workflowsInfo returns the jobs that've been resumed
634     * @throws DagEngineException if the jobs cloud not be resumed
635     */
636    public WorkflowsInfo resumeJobs(String filter, int start, int len) throws DagEngineException {
637        try {
638            Map<String, List<String>> filterList = parseFilter(filter);
639            WorkflowsInfo workflowsInfo = new BulkWorkflowXCommand(filterList, start, len, OperationType.Resume).call();
640            if (workflowsInfo == null) {
641                return new WorkflowsInfo(new ArrayList<WorkflowJobBean>(), 0, 0, 0);
642            }
643            return workflowsInfo;
644        }
645        catch (CommandException ex) {
646            throw new DagEngineException(ex);
647        }
648    }
649}