001    /**
002     * Copyright (c) 2010 Yahoo! Inc. All rights reserved.
003     * Licensed under the Apache License, Version 2.0 (the "License");
004     * you may not use this file except in compliance with the License.
005     * You may obtain a copy of the License at
006     *
007     *   http://www.apache.org/licenses/LICENSE-2.0
008     *
009     *  Unless required by applicable law or agreed to in writing, software
010     *  distributed under the License is distributed on an "AS IS" BASIS,
011     *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
012     *  See the License for the specific language governing permissions and
013     *  limitations under the License. See accompanying LICENSE file.
014     */
015    package org.apache.oozie.command.coord;
016    
017    import java.io.IOException;
018    import java.io.InputStreamReader;
019    import java.io.Reader;
020    import java.io.StringReader;
021    import java.io.StringWriter;
022    import java.net.URI;
023    import java.net.URISyntaxException;
024    import java.util.ArrayList;
025    import java.util.Date;
026    import java.util.HashMap;
027    import java.util.HashSet;
028    import java.util.List;
029    import java.util.Map;
030    import java.util.Set;
031    import java.util.TreeSet;
032    
033    import javax.xml.transform.stream.StreamSource;
034    import javax.xml.validation.Validator;
035    
036    import org.apache.hadoop.conf.Configuration;
037    import org.apache.hadoop.fs.FileSystem;
038    import org.apache.hadoop.fs.Path;
039    import org.apache.oozie.CoordinatorJobBean;
040    import org.apache.oozie.ErrorCode;
041    import org.apache.oozie.client.CoordinatorJob;
042    import org.apache.oozie.client.Job;
043    import org.apache.oozie.client.OozieClient;
044    import org.apache.oozie.client.CoordinatorJob.Execution;
045    import org.apache.oozie.command.CommandException;
046    import org.apache.oozie.command.SubmitTransitionXCommand;
047    import org.apache.oozie.command.bundle.BundleStatusUpdateXCommand;
048    import org.apache.oozie.coord.CoordELEvaluator;
049    import org.apache.oozie.coord.CoordELFunctions;
050    import org.apache.oozie.coord.CoordinatorJobException;
051    import org.apache.oozie.coord.TimeUnit;
052    import org.apache.oozie.executor.jpa.CoordJobInsertJPAExecutor;
053    import org.apache.oozie.executor.jpa.JPAExecutorException;
054    import org.apache.oozie.service.DagXLogInfoService;
055    import org.apache.oozie.service.HadoopAccessorException;
056    import org.apache.oozie.service.HadoopAccessorService;
057    import org.apache.oozie.service.JPAService;
058    import org.apache.oozie.service.SchemaService;
059    import org.apache.oozie.service.Service;
060    import org.apache.oozie.service.Services;
061    import org.apache.oozie.service.UUIDService;
062    import org.apache.oozie.service.WorkflowAppService;
063    import org.apache.oozie.service.SchemaService.SchemaName;
064    import org.apache.oozie.service.UUIDService.ApplicationType;
065    import org.apache.oozie.util.DateUtils;
066    import org.apache.oozie.util.ELEvaluator;
067    import org.apache.oozie.util.IOUtils;
068    import org.apache.oozie.util.InstrumentUtils;
069    import org.apache.oozie.util.LogUtils;
070    import org.apache.oozie.util.ParamChecker;
071    import org.apache.oozie.util.PropertiesUtils;
072    import org.apache.oozie.util.XConfiguration;
073    import org.apache.oozie.util.XLog;
074    import org.apache.oozie.util.XmlUtils;
075    import org.jdom.Attribute;
076    import org.jdom.Element;
077    import org.jdom.JDOMException;
078    import org.jdom.Namespace;
079    import org.xml.sax.SAXException;
080    
081    /**
082     * This class provides the functionalities to resolve a coordinator job XML and write the job information into a DB
083     * table.
084     * <p/>
085     * Specifically it performs the following functions: 1. Resolve all the variables or properties using job
086     * configurations. 2. Insert all datasets definition as part of the <data-in> and <data-out> tags. 3. Validate the XML
087     * at runtime.
088     */
089    public class CoordSubmitXCommand extends SubmitTransitionXCommand {
090    
091        private Configuration conf;
092        private final String authToken;
093        private final String bundleId;
094        private final String coordName;
095        private boolean dryrun;
096        private JPAService jpaService = null;
097        private CoordinatorJob.Status prevStatus = CoordinatorJob.Status.PREP;
098    
099        public static final String CONFIG_DEFAULT = "coord-config-default.xml";
100        public static final String COORDINATOR_XML_FILE = "coordinator.xml";
101    
102        private static final Set<String> DISALLOWED_USER_PROPERTIES = new HashSet<String>();
103        private static final Set<String> DISALLOWED_DEFAULT_PROPERTIES = new HashSet<String>();
104    
105        private CoordinatorJobBean coordJob = null;
106        /**
107         * Default timeout for normal jobs, in minutes, after which coordinator input check will timeout
108         */
109        public static final String CONF_DEFAULT_TIMEOUT_NORMAL = Service.CONF_PREFIX + "coord.normal.default.timeout";
110    
111        public static final String CONF_DEFAULT_CONCURRENCY = Service.CONF_PREFIX + "coord.default.concurrency";
112    
113        public static final String CONF_DEFAULT_THROTTLE = Service.CONF_PREFIX + "coord.default.throttle";
114    
115        public static final String CONF_MAT_THROTTLING_FACTOR = Service.CONF_PREFIX
116                + "coord.materialization.throttling.factor";
117    
118        /**
119         * Default MAX timeout in minutes, after which coordinator input check will timeout
120         */
121        public static final String CONF_DEFAULT_MAX_TIMEOUT = Service.CONF_PREFIX + "coord.default.max.timeout";
122    
123    
124        public static final String CONF_QUEUE_SIZE = Service.CONF_PREFIX + "CallableQueueService.queue.size";
125    
126        private ELEvaluator evalFreq = null;
127        private ELEvaluator evalNofuncs = null;
128        private ELEvaluator evalData = null;
129        private ELEvaluator evalInst = null;
130        private ELEvaluator evalSla = null;
131    
132        static {
133            String[] badUserProps = { PropertiesUtils.YEAR, PropertiesUtils.MONTH, PropertiesUtils.DAY,
134                    PropertiesUtils.HOUR, PropertiesUtils.MINUTE, PropertiesUtils.DAYS, PropertiesUtils.HOURS,
135                    PropertiesUtils.MINUTES, PropertiesUtils.KB, PropertiesUtils.MB, PropertiesUtils.GB,
136                    PropertiesUtils.TB, PropertiesUtils.PB, PropertiesUtils.RECORDS, PropertiesUtils.MAP_IN,
137                    PropertiesUtils.MAP_OUT, PropertiesUtils.REDUCE_IN, PropertiesUtils.REDUCE_OUT, PropertiesUtils.GROUPS };
138            PropertiesUtils.createPropertySet(badUserProps, DISALLOWED_USER_PROPERTIES);
139    
140            String[] badDefaultProps = { PropertiesUtils.HADOOP_USER, PropertiesUtils.HADOOP_UGI,
141                    WorkflowAppService.HADOOP_JT_KERBEROS_NAME, WorkflowAppService.HADOOP_NN_KERBEROS_NAME };
142            PropertiesUtils.createPropertySet(badUserProps, DISALLOWED_DEFAULT_PROPERTIES);
143            PropertiesUtils.createPropertySet(badDefaultProps, DISALLOWED_DEFAULT_PROPERTIES);
144        }
145    
146        /**
147         * Constructor to create the Coordinator Submit Command.
148         *
149         * @param conf : Configuration for Coordinator job
150         * @param authToken : To be used for authentication
151         */
152        public CoordSubmitXCommand(Configuration conf, String authToken) {
153            super("coord_submit", "coord_submit", 1);
154            this.conf = ParamChecker.notNull(conf, "conf");
155            this.authToken = ParamChecker.notEmpty(authToken, "authToken");
156            this.bundleId = null;
157            this.coordName = null;
158        }
159    
160        /**
161         * Constructor to create the Coordinator Submit Command by bundle job.
162         *
163         * @param conf : Configuration for Coordinator job
164         * @param authToken : To be used for authentication
165         * @param bundleId : bundle id
166         * @param coordName : coord name
167         */
168        public CoordSubmitXCommand(Configuration conf, String authToken, String bundleId, String coordName) {
169            super("coord_submit", "coord_submit", 1);
170            this.conf = ParamChecker.notNull(conf, "conf");
171            this.authToken = ParamChecker.notEmpty(authToken, "authToken");
172            this.bundleId = ParamChecker.notEmpty(bundleId, "bundleId");
173            this.coordName = ParamChecker.notEmpty(coordName, "coordName");
174        }
175    
176        /**
177         * Constructor to create the Coordinator Submit Command.
178         *
179         * @param dryrun : if dryrun
180         * @param conf : Configuration for Coordinator job
181         * @param authToken : To be used for authentication
182         */
183        public CoordSubmitXCommand(boolean dryrun, Configuration conf, String authToken) {
184            this(conf, authToken);
185            this.dryrun = dryrun;
186        }
187    
188        /* (non-Javadoc)
189         * @see org.apache.oozie.command.XCommand#execute()
190         */
191        @Override
192        protected String submit() throws CommandException {
193            String jobId = null;
194            LOG.info("STARTED Coordinator Submit");
195            InstrumentUtils.incrJobCounter(getName(), 1, getInstrumentation());
196    
197            boolean exceptionOccured = false;
198            try {
199                mergeDefaultConfig();
200    
201                String appXml = readAndValidateXml();
202                coordJob.setOrigJobXml(appXml);
203                LOG.debug("jobXml after initial validation " + XmlUtils.prettyPrint(appXml).toString());
204    
205                String appNamespace = readAppNamespace(appXml);
206                coordJob.setAppNamespace(appNamespace);
207    
208                appXml = XmlUtils.removeComments(appXml);
209                initEvaluators();
210                Element eJob = basicResolveAndIncludeDS(appXml, conf, coordJob);
211                LOG.debug("jobXml after all validation " + XmlUtils.prettyPrint(eJob).toString());
212    
213                jobId = storeToDB(eJob, coordJob);
214                // log job info for coordinator job
215                LogUtils.setLogInfo(coordJob, logInfo);
216                LOG = XLog.resetPrefix(LOG);
217    
218                if (!dryrun) {
219                    // submit a command to materialize jobs for the next 1 hour (3600 secs)
220                    // so we don't wait 10 mins for the Service to run.
221                    queue(new CoordMaterializeTransitionXCommand(jobId, 3600), 100);
222                }
223                else {
224                    Date startTime = coordJob.getStartTime();
225                    long startTimeMilli = startTime.getTime();
226                    long endTimeMilli = startTimeMilli + (3600 * 1000);
227                    Date jobEndTime = coordJob.getEndTime();
228                    Date endTime = new Date(endTimeMilli);
229                    if (endTime.compareTo(jobEndTime) > 0) {
230                        endTime = jobEndTime;
231                    }
232                    jobId = coordJob.getId();
233                    LOG.info("[" + jobId + "]: Update status to RUNNING");
234                    coordJob.setStatus(Job.Status.RUNNING);
235                    coordJob.setPending();
236                    CoordActionMaterializeCommand coordActionMatCom = new CoordActionMaterializeCommand(jobId, startTime,
237                            endTime);
238                    Configuration jobConf = null;
239                    try {
240                        jobConf = new XConfiguration(new StringReader(coordJob.getConf()));
241                    }
242                    catch (IOException e1) {
243                        LOG.warn("Configuration parse error. read from DB :" + coordJob.getConf(), e1);
244                    }
245                    String action = coordActionMatCom.materializeJobs(true, coordJob, jobConf, null);
246                    String output = coordJob.getJobXml() + System.getProperty("line.separator")
247                    + "***actions for instance***" + action;
248                    return output;
249                }
250            }
251            catch (CoordinatorJobException cex) {
252                exceptionOccured = true;
253                LOG.warn("ERROR:  ", cex);
254                throw new CommandException(cex);
255            }
256            catch (IllegalArgumentException iex) {
257                exceptionOccured = true;
258                LOG.warn("ERROR:  ", iex);
259                throw new CommandException(ErrorCode.E1003, iex);
260            }
261            catch (Exception ex) {
262                exceptionOccured = true;
263                LOG.warn("ERROR:  ", ex);
264                throw new CommandException(ErrorCode.E0803, ex);
265            }
266            finally {
267                if (exceptionOccured) {
268                    if(coordJob.getId() == null || coordJob.getId().equalsIgnoreCase("")){
269                        coordJob.setStatus(CoordinatorJob.Status.FAILED);
270                        coordJob.resetPending();
271                    }
272                }
273            }
274    
275            LOG.info("ENDED Coordinator Submit jobId=" + jobId);
276            return jobId;
277        }
278    
279        /**
280         * Read the application XML and validate against coordinator Schema
281         *
282         * @return validated coordinator XML
283         * @throws CoordinatorJobException thrown if unable to read or validate coordinator xml
284         */
285        private String readAndValidateXml() throws CoordinatorJobException {
286            String appPath = ParamChecker.notEmpty(conf.get(OozieClient.COORDINATOR_APP_PATH),
287                    OozieClient.COORDINATOR_APP_PATH);
288            String coordXml = readDefinition(appPath);
289            validateXml(coordXml);
290            return coordXml;
291        }
292    
293        /**
294         * Validate against Coordinator XSD file
295         *
296         * @param xmlContent : Input coordinator xml
297         * @throws CoordinatorJobException thrown if unable to validate coordinator xml
298         */
299        private void validateXml(String xmlContent) throws CoordinatorJobException {
300            javax.xml.validation.Schema schema = Services.get().get(SchemaService.class).getSchema(SchemaName.COORDINATOR);
301            Validator validator = schema.newValidator();
302            try {
303                validator.validate(new StreamSource(new StringReader(xmlContent)));
304            }
305            catch (SAXException ex) {
306                LOG.warn("SAXException :", ex);
307                throw new CoordinatorJobException(ErrorCode.E0701, ex.getMessage(), ex);
308            }
309            catch (IOException ex) {
310                LOG.warn("IOException :", ex);
311                throw new CoordinatorJobException(ErrorCode.E0702, ex.getMessage(), ex);
312            }
313        }
314    
315        /**
316         * Read the application XML schema namespace
317         *
318         * @param xmlContent input coordinator xml
319         * @return app xml namespace
320         * @throws CoordinatorJobException
321         */
322        private String readAppNamespace(String xmlContent) throws CoordinatorJobException {
323            try {
324                Element coordXmlElement = XmlUtils.parseXml(xmlContent);
325                Namespace ns = coordXmlElement.getNamespace();
326                if (ns != null && bundleId != null && ns.getURI().equals(SchemaService.COORDINATOR_NAMESPACE_URI_1)) {
327                    throw new CoordinatorJobException(ErrorCode.E1319, "bundle app can not submit coordinator namespace "
328                            + SchemaService.COORDINATOR_NAMESPACE_URI_1 + ", please use 0.2 or later");
329                }
330                if (ns != null) {
331                    return ns.getURI();
332                }
333                else {
334                    throw new CoordinatorJobException(ErrorCode.E0700, "the application xml namespace is not given");
335                }
336            }
337            catch (JDOMException ex) {
338                LOG.warn("JDOMException :", ex);
339                throw new CoordinatorJobException(ErrorCode.E0700, ex.getMessage(), ex);
340            }
341        }
342    
343        /**
344         * Merge default configuration with user-defined configuration.
345         *
346         * @throws CommandException thrown if failed to read or merge configurations
347         */
348        protected void mergeDefaultConfig() throws CommandException {
349            Path configDefault = null;
350            try {
351                String coordAppPathStr = conf.get(OozieClient.COORDINATOR_APP_PATH);
352                Path coordAppPath = new Path(coordAppPathStr);
353                String user = ParamChecker.notEmpty(conf.get(OozieClient.USER_NAME), OozieClient.USER_NAME);
354                String group = ParamChecker.notEmpty(conf.get(OozieClient.GROUP_NAME), OozieClient.GROUP_NAME);
355                FileSystem fs = Services.get().get(HadoopAccessorService.class).createFileSystem(user, group,
356                        coordAppPath.toUri(), new Configuration());
357    
358                // app path could be a directory
359                if (!fs.isFile(coordAppPath)) {
360                    configDefault = new Path(coordAppPath, CONFIG_DEFAULT);
361                } else {
362                    configDefault = new Path(coordAppPath.getParent(), CONFIG_DEFAULT);
363                }
364    
365                if (fs.exists(configDefault)) {
366                    Configuration defaultConf = new XConfiguration(fs.open(configDefault));
367                    PropertiesUtils.checkDisallowedProperties(defaultConf, DISALLOWED_DEFAULT_PROPERTIES);
368                    XConfiguration.injectDefaults(defaultConf, conf);
369                }
370                else {
371                    LOG.info("configDefault Doesn't exist " + configDefault);
372                }
373                PropertiesUtils.checkDisallowedProperties(conf, DISALLOWED_USER_PROPERTIES);
374    
375                // Resolving all variables in the job properties.
376                // This ensures the Hadoop Configuration semantics is preserved.
377                XConfiguration resolvedVarsConf = new XConfiguration();
378                for (Map.Entry<String, String> entry : conf) {
379                    resolvedVarsConf.set(entry.getKey(), conf.get(entry.getKey()));
380                }
381                conf = resolvedVarsConf;
382            }
383            catch (IOException e) {
384                throw new CommandException(ErrorCode.E0702, e.getMessage() + " : Problem reading default config "
385                        + configDefault, e);
386            }
387            catch (HadoopAccessorException e) {
388                throw new CommandException(e);
389            }
390            LOG.debug("Merged CONF :" + XmlUtils.prettyPrint(conf).toString());
391        }
392    
393        /**
394         * The method resolve all the variables that are defined in configuration. It also include the data set definition
395         * from dataset file into XML.
396         *
397         * @param appXml : Original job XML
398         * @param conf : Configuration of the job
399         * @param coordJob : Coordinator job bean to be populated.
400         * @return Resolved and modified job XML element.
401         * @throws CoordinatorJobException thrown if failed to resolve basic entities or include referred datasets
402         * @throws Exception thrown if failed to resolve basic entities or include referred datasets
403         */
404        public Element basicResolveAndIncludeDS(String appXml, Configuration conf, CoordinatorJobBean coordJob)
405        throws CoordinatorJobException, Exception {
406            Element basicResolvedApp = resolveInitial(conf, appXml, coordJob);
407            includeDataSets(basicResolvedApp, conf);
408            return basicResolvedApp;
409        }
410    
411        /**
412         * Insert data set into data-in and data-out tags.
413         *
414         * @param eAppXml : coordinator application XML
415         * @param eDatasets : DataSet XML
416         */
417        @SuppressWarnings("unchecked")
418        private void insertDataSet(Element eAppXml, Element eDatasets) {
419            // Adding DS definition in the coordinator XML
420            Element inputList = eAppXml.getChild("input-events", eAppXml.getNamespace());
421            if (inputList != null) {
422                for (Element dataIn : (List<Element>) inputList.getChildren("data-in", eAppXml.getNamespace())) {
423                    Element eDataset = findDataSet(eDatasets, dataIn.getAttributeValue("dataset"));
424                    dataIn.getContent().add(0, eDataset);
425                }
426            }
427            Element outputList = eAppXml.getChild("output-events", eAppXml.getNamespace());
428            if (outputList != null) {
429                for (Element dataOut : (List<Element>) outputList.getChildren("data-out", eAppXml.getNamespace())) {
430                    Element eDataset = findDataSet(eDatasets, dataOut.getAttributeValue("dataset"));
431                    dataOut.getContent().add(0, eDataset);
432                }
433            }
434        }
435    
436        /**
437         * Find a specific dataset from a list of Datasets.
438         *
439         * @param eDatasets : List of data sets
440         * @param name : queried data set name
441         * @return one Dataset element. otherwise throw Exception
442         */
443        @SuppressWarnings("unchecked")
444        private static Element findDataSet(Element eDatasets, String name) {
445            for (Element eDataset : (List<Element>) eDatasets.getChildren("dataset", eDatasets.getNamespace())) {
446                if (eDataset.getAttributeValue("name").equals(name)) {
447                    eDataset = (Element) eDataset.clone();
448                    eDataset.detach();
449                    return eDataset;
450                }
451            }
452            throw new RuntimeException("undefined dataset: " + name);
453        }
454    
455        /**
456         * Initialize all the required EL Evaluators.
457         */
458        protected void initEvaluators() {
459            evalFreq = CoordELEvaluator.createELEvaluatorForGroup(conf, "coord-job-submit-freq");
460            evalNofuncs = CoordELEvaluator.createELEvaluatorForGroup(conf, "coord-job-submit-nofuncs");
461            evalInst = CoordELEvaluator.createELEvaluatorForGroup(conf, "coord-job-submit-instances");
462            evalSla = CoordELEvaluator.createELEvaluatorForGroup(conf, "coord-sla-submit");
463        }
464    
465        /**
466         * Resolve basic entities using job Configuration.
467         *
468         * @param conf :Job configuration
469         * @param appXml : Original job XML
470         * @param coordJob : Coordinator job bean to be populated.
471         * @return Resolved job XML element.
472         * @throws CoordinatorJobException thrown if failed to resolve basic entities
473         * @throws Exception thrown if failed to resolve basic entities
474         */
475        @SuppressWarnings("unchecked")
476        protected Element resolveInitial(Configuration conf, String appXml, CoordinatorJobBean coordJob)
477        throws CoordinatorJobException, Exception {
478            Element eAppXml = XmlUtils.parseXml(appXml);
479            // job's main attributes
480            // frequency
481            String val = resolveAttribute("frequency", eAppXml, evalFreq);
482            int ival = ParamChecker.checkInteger(val, "frequency");
483            ParamChecker.checkGTZero(ival, "frequency");
484            coordJob.setFrequency(ival);
485            TimeUnit tmp = (evalFreq.getVariable("timeunit") == null) ? TimeUnit.MINUTE : ((TimeUnit) evalFreq
486                    .getVariable("timeunit"));
487            addAnAttribute("freq_timeunit", eAppXml, tmp.toString());
488            // TimeUnit
489            coordJob.setTimeUnit(CoordinatorJob.Timeunit.valueOf(tmp.toString()));
490            // End Of Duration
491            tmp = evalFreq.getVariable("endOfDuration") == null ? TimeUnit.NONE : ((TimeUnit) evalFreq
492                    .getVariable("endOfDuration"));
493            addAnAttribute("end_of_duration", eAppXml, tmp.toString());
494            // coordJob.setEndOfDuration(tmp) // TODO: Add new attribute in Job bean
495    
496            // start time
497            val = resolveAttribute("start", eAppXml, evalNofuncs);
498            ParamChecker.checkUTC(val, "start");
499            coordJob.setStartTime(DateUtils.parseDateUTC(val));
500            // end time
501            val = resolveAttribute("end", eAppXml, evalNofuncs);
502            ParamChecker.checkUTC(val, "end");
503            coordJob.setEndTime(DateUtils.parseDateUTC(val));
504            // Time zone
505            val = resolveAttribute("timezone", eAppXml, evalNofuncs);
506            ParamChecker.checkTimeZone(val, "timezone");
507            coordJob.setTimeZone(val);
508    
509            // controls
510            val = resolveTagContents("timeout", eAppXml.getChild("controls", eAppXml.getNamespace()), evalNofuncs);
511            if (val == "") {
512                val = Services.get().getConf().get(CONF_DEFAULT_TIMEOUT_NORMAL);
513            }
514    
515            ival = ParamChecker.checkInteger(val, "timeout");
516            if (ival < 0 || ival > Services.get().getConf().getInt(CONF_DEFAULT_MAX_TIMEOUT, 129600)) {
517                ival = Services.get().getConf().getInt(CONF_DEFAULT_MAX_TIMEOUT, 129600);
518            }
519            coordJob.setTimeout(ival);
520    
521            val = resolveTagContents("concurrency", eAppXml.getChild("controls", eAppXml.getNamespace()), evalNofuncs);
522            if (val == null || val.isEmpty()) {
523                val = Services.get().getConf().get(CONF_DEFAULT_CONCURRENCY, "-1");
524            }
525            ival = ParamChecker.checkInteger(val, "concurrency");
526            coordJob.setConcurrency(ival);
527    
528            val = resolveTagContents("throttle", eAppXml.getChild("controls", eAppXml.getNamespace()), evalNofuncs);
529            if (val == null || val.isEmpty()) {
530                int defaultThrottle = Services.get().getConf().getInt(CONF_DEFAULT_THROTTLE, 12);
531                ival = defaultThrottle;
532            }
533            else {
534                ival = ParamChecker.checkInteger(val, "throttle");
535            }
536            int maxQueue = Services.get().getConf().getInt(CONF_QUEUE_SIZE, 10000);
537            float factor = Services.get().getConf().getFloat(CONF_MAT_THROTTLING_FACTOR, 0.10f);
538            int maxThrottle = (int) (maxQueue * factor);
539            if (ival > maxThrottle || ival < 1) {
540                ival = maxThrottle;
541            }
542            LOG.debug("max throttle " + ival);
543            coordJob.setMatThrottling(ival);
544    
545            val = resolveTagContents("execution", eAppXml.getChild("controls", eAppXml.getNamespace()), evalNofuncs);
546            if (val == "") {
547                val = Execution.FIFO.toString();
548            }
549            coordJob.setExecution(Execution.valueOf(val));
550            String[] acceptedVals = { Execution.LIFO.toString(), Execution.FIFO.toString(), Execution.LAST_ONLY.toString() };
551            ParamChecker.isMember(val, acceptedVals, "execution");
552    
553            // datasets
554            resolveTagContents("include", eAppXml.getChild("datasets", eAppXml.getNamespace()), evalNofuncs);
555            // for each data set
556            resolveDataSets(eAppXml);
557            HashMap<String, String> dataNameList = new HashMap<String, String>();
558            resolveIOEvents(eAppXml, dataNameList);
559    
560            resolveTagContents("app-path", eAppXml.getChild("action", eAppXml.getNamespace()).getChild("workflow",
561                    eAppXml.getNamespace()), evalNofuncs);
562            // TODO: If action or workflow tag is missing, NullPointerException will
563            // occur
564            Element configElem = eAppXml.getChild("action", eAppXml.getNamespace()).getChild("workflow",
565                    eAppXml.getNamespace()).getChild("configuration", eAppXml.getNamespace());
566            evalData = CoordELEvaluator.createELEvaluatorForDataEcho(conf, "coord-job-submit-data", dataNameList);
567            if (configElem != null) {
568                for (Element propElem : (List<Element>) configElem.getChildren("property", configElem.getNamespace())) {
569                    resolveTagContents("name", propElem, evalData);
570                    // Want to check the data-integrity but don't want to modify the
571                    // XML
572                    // for properties only
573                    Element tmpProp = (Element) propElem.clone();
574                    resolveTagContents("value", tmpProp, evalData);
575                }
576            }
577            resolveSLA(eAppXml, coordJob);
578            return eAppXml;
579        }
580    
581        /**
582         * Resolve SLA events
583         *
584         * @param eAppXml job XML
585         * @param coordJob coordinator job bean
586         * @throws CommandException thrown if failed to resolve sla events
587         */
588        private void resolveSLA(Element eAppXml, CoordinatorJobBean coordJob) throws CommandException {
589            Element eSla = eAppXml.getChild("action", eAppXml.getNamespace()).getChild("info",
590                    Namespace.getNamespace(SchemaService.SLA_NAME_SPACE_URI));
591    
592            if (eSla != null) {
593                String slaXml = XmlUtils.prettyPrint(eSla).toString();
594                try {
595                    // EL evaluation
596                    slaXml = evalSla.evaluate(slaXml, String.class);
597                    // Validate against semantic SXD
598                    XmlUtils.validateData(slaXml, SchemaName.SLA_ORIGINAL);
599                }
600                catch (Exception e) {
601                    throw new CommandException(ErrorCode.E1004, "Validation ERROR :" + e.getMessage(), e);
602                }
603            }
604        }
605    
606        /**
607         * Resolve input-events/data-in and output-events/data-out tags.
608         *
609         * @param eJob : Job element
610         * @throws CoordinatorJobException thrown if failed to resolve input and output events
611         */
612        @SuppressWarnings("unchecked")
613        private void resolveIOEvents(Element eJobOrg, HashMap<String, String> dataNameList) throws CoordinatorJobException {
614            // Resolving input-events/data-in
615            // Clone the job and don't update anything in the original
616            Element eJob = (Element) eJobOrg.clone();
617            Element inputList = eJob.getChild("input-events", eJob.getNamespace());
618            if (inputList != null) {
619                TreeSet<String> eventNameSet = new TreeSet<String>();
620                for (Element dataIn : (List<Element>) inputList.getChildren("data-in", eJob.getNamespace())) {
621                    String dataInName = dataIn.getAttributeValue("name");
622                    dataNameList.put(dataInName, "data-in");
623                    // check whether there is any duplicate data-in name
624                    if (eventNameSet.contains(dataInName)) {
625                        throw new RuntimeException("Duplicate dataIn name " + dataInName);
626                    }
627                    else {
628                        eventNameSet.add(dataInName);
629                    }
630                    resolveTagContents("instance", dataIn, evalInst);
631                    resolveTagContents("start-instance", dataIn, evalInst);
632                    resolveTagContents("end-instance", dataIn, evalInst);
633                }
634            }
635            // Resolving output-events/data-out
636            Element outputList = eJob.getChild("output-events", eJob.getNamespace());
637            if (outputList != null) {
638                TreeSet<String> eventNameSet = new TreeSet<String>();
639                for (Element dataOut : (List<Element>) outputList.getChildren("data-out", eJob.getNamespace())) {
640                    String dataOutName = dataOut.getAttributeValue("name");
641                    dataNameList.put(dataOutName, "data-out");
642                    // check whether there is any duplicate data-out name
643                    if (eventNameSet.contains(dataOutName)) {
644                        throw new RuntimeException("Duplicate dataIn name " + dataOutName);
645                    }
646                    else {
647                        eventNameSet.add(dataOutName);
648                    }
649                    resolveTagContents("instance", dataOut, evalInst);
650                }
651            }
652    
653        }
654    
655        /**
656         * Add an attribute into XML element.
657         *
658         * @param attrName :attribute name
659         * @param elem : Element to add attribute
660         * @param value :Value of attribute
661         */
662        private void addAnAttribute(String attrName, Element elem, String value) {
663            elem.setAttribute(attrName, value);
664        }
665    
666        /**
667         * Resolve datasets using job configuration.
668         *
669         * @param eAppXml : Job Element XML
670         * @throws Exception thrown if failed to resolve datasets
671         */
672        @SuppressWarnings("unchecked")
673        private void resolveDataSets(Element eAppXml) throws Exception {
674            Element datasetList = eAppXml.getChild("datasets", eAppXml.getNamespace());
675            if (datasetList != null) {
676    
677                List<Element> dsElems = datasetList.getChildren("dataset", eAppXml.getNamespace());
678                resolveDataSets(dsElems);
679                resolveTagContents("app-path", eAppXml.getChild("action", eAppXml.getNamespace()).getChild("workflow",
680                        eAppXml.getNamespace()), evalNofuncs);
681            }
682        }
683    
684        /**
685         * Resolve datasets using job configuration.
686         *
687         * @param dsElems : Data set XML element.
688         * @throws CoordinatorJobException thrown if failed to resolve datasets
689         */
690        private void resolveDataSets(List<Element> dsElems) throws CoordinatorJobException {
691            for (Element dsElem : dsElems) {
692                // Setting up default TimeUnit and EndOFDuraion
693                evalFreq.setVariable("timeunit", TimeUnit.MINUTE);
694                evalFreq.setVariable("endOfDuration", TimeUnit.NONE);
695    
696                String val = resolveAttribute("frequency", dsElem, evalFreq);
697                int ival = ParamChecker.checkInteger(val, "frequency");
698                ParamChecker.checkGTZero(ival, "frequency");
699                addAnAttribute("freq_timeunit", dsElem, evalFreq.getVariable("timeunit") == null ? TimeUnit.MINUTE
700                        .toString() : ((TimeUnit) evalFreq.getVariable("timeunit")).toString());
701                addAnAttribute("end_of_duration", dsElem, evalFreq.getVariable("endOfDuration") == null ? TimeUnit.NONE
702                        .toString() : ((TimeUnit) evalFreq.getVariable("endOfDuration")).toString());
703                val = resolveAttribute("initial-instance", dsElem, evalNofuncs);
704                ParamChecker.checkUTC(val, "initial-instance");
705                val = resolveAttribute("timezone", dsElem, evalNofuncs);
706                ParamChecker.checkTimeZone(val, "timezone");
707                resolveTagContents("uri-template", dsElem, evalNofuncs);
708                resolveTagContents("done-flag", dsElem, evalNofuncs);
709            }
710        }
711    
712        /**
713         * Resolve the content of a tag.
714         *
715         * @param tagName : Tag name of job XML i.e. <timeout> 10 </timeout>
716         * @param elem : Element where the tag exists.
717         * @param eval : EL evealuator
718         * @return Resolved tag content.
719         * @throws CoordinatorJobException thrown if failed to resolve tag content
720         */
721        @SuppressWarnings("unchecked")
722        private String resolveTagContents(String tagName, Element elem, ELEvaluator eval) throws CoordinatorJobException {
723            String ret = "";
724            if (elem != null) {
725                for (Element tagElem : (List<Element>) elem.getChildren(tagName, elem.getNamespace())) {
726                    if (tagElem != null) {
727                        String updated;
728                        try {
729                            updated = CoordELFunctions.evalAndWrap(eval, tagElem.getText().trim());
730    
731                        }
732                        catch (Exception e) {
733                            throw new CoordinatorJobException(ErrorCode.E1004, e.getMessage(), e);
734                        }
735                        tagElem.removeContent();
736                        tagElem.addContent(updated);
737                        ret += updated;
738                    }
739                }
740            }
741            return ret;
742        }
743    
744        /**
745         * Resolve an attribute value.
746         *
747         * @param attrName : Attribute name.
748         * @param elem : XML Element where attribute is defiend
749         * @param eval : ELEvaluator used to resolve
750         * @return Resolved attribute value
751         * @throws CoordinatorJobException thrown if failed to resolve an attribute value
752         */
753        private String resolveAttribute(String attrName, Element elem, ELEvaluator eval) throws CoordinatorJobException {
754            Attribute attr = elem.getAttribute(attrName);
755            String val = null;
756            if (attr != null) {
757                try {
758                    val = CoordELFunctions.evalAndWrap(eval, attr.getValue().trim());
759    
760                }
761                catch (Exception e) {
762                    throw new CoordinatorJobException(ErrorCode.E1004, e.getMessage(), e);
763                }
764                attr.setValue(val);
765            }
766            return val;
767        }
768    
769        /**
770         * Include referred datasets into XML.
771         *
772         * @param resolvedXml : Job XML element.
773         * @param conf : Job configuration
774         * @throws CoordinatorJobException thrown if failed to include referred datasets into XML
775         */
776        @SuppressWarnings("unchecked")
777        protected void includeDataSets(Element resolvedXml, Configuration conf) throws CoordinatorJobException {
778            Element datasets = resolvedXml.getChild("datasets", resolvedXml.getNamespace());
779            Element allDataSets = new Element("all_datasets", resolvedXml.getNamespace());
780            List<String> dsList = new ArrayList<String>();
781            if (datasets != null) {
782                for (Element includeElem : (List<Element>) datasets.getChildren("include", datasets.getNamespace())) {
783                    String incDSFile = includeElem.getTextTrim();
784                    includeOneDSFile(incDSFile, dsList, allDataSets, datasets.getNamespace());
785                }
786                for (Element e : (List<Element>) datasets.getChildren("dataset", datasets.getNamespace())) {
787                    String dsName = e.getAttributeValue("name");
788                    if (dsList.contains(dsName)) {// Override with this DS
789                        // Remove old DS
790                        removeDataSet(allDataSets, dsName);
791                    }
792                    else {
793                        dsList.add(dsName);
794                    }
795                    allDataSets.addContent((Element) e.clone());
796                }
797            }
798            insertDataSet(resolvedXml, allDataSets);
799            resolvedXml.removeChild("datasets", resolvedXml.getNamespace());
800        }
801    
802        /**
803         * Include one dataset file.
804         *
805         * @param incDSFile : Include data set filename.
806         * @param dsList :List of dataset names to verify the duplicate.
807         * @param allDataSets : Element that includes all dataset definitions.
808         * @param dsNameSpace : Data set name space
809         * @throws CoordinatorJobException thrown if failed to include one dataset file
810         */
811        @SuppressWarnings("unchecked")
812        private void includeOneDSFile(String incDSFile, List<String> dsList, Element allDataSets, Namespace dsNameSpace)
813        throws CoordinatorJobException {
814            Element tmpDataSets = null;
815            try {
816                String dsXml = readDefinition(incDSFile);
817                LOG.debug("DSFILE :" + incDSFile + "\n" + dsXml);
818                tmpDataSets = XmlUtils.parseXml(dsXml);
819            }
820            catch (JDOMException e) {
821                LOG.warn("Error parsing included dataset [{0}].  Message [{1}]", incDSFile, e.getMessage());
822                throw new CoordinatorJobException(ErrorCode.E0700, e.getMessage());
823            }
824            resolveDataSets(tmpDataSets.getChildren("dataset"));
825            for (Element e : (List<Element>) tmpDataSets.getChildren("dataset")) {
826                String dsName = e.getAttributeValue("name");
827                if (dsList.contains(dsName)) {
828                    throw new RuntimeException("Duplicate Dataset " + dsName);
829                }
830                dsList.add(dsName);
831                Element tmp = (Element) e.clone();
832                // TODO: Don't like to over-write the external/include DS's namespace
833                tmp.setNamespace(dsNameSpace);
834                tmp.getChild("uri-template").setNamespace(dsNameSpace);
835                if (e.getChild("done-flag") != null) {
836                    tmp.getChild("done-flag").setNamespace(dsNameSpace);
837                }
838                allDataSets.addContent(tmp);
839            }
840            // nested include
841            for (Element includeElem : (List<Element>) tmpDataSets.getChildren("include", tmpDataSets.getNamespace())) {
842                String incFile = includeElem.getTextTrim();
843                includeOneDSFile(incFile, dsList, allDataSets, dsNameSpace);
844            }
845        }
846    
847        /**
848         * Remove a dataset from a list of dataset.
849         *
850         * @param eDatasets : List of dataset
851         * @param name : Dataset name to be removed.
852         */
853        @SuppressWarnings("unchecked")
854        private static void removeDataSet(Element eDatasets, String name) {
855            for (Element eDataset : (List<Element>) eDatasets.getChildren("dataset", eDatasets.getNamespace())) {
856                if (eDataset.getAttributeValue("name").equals(name)) {
857                    eDataset.detach();
858                }
859            }
860            throw new RuntimeException("undefined dataset: " + name);
861        }
862    
863        /**
864         * Read coordinator definition.
865         *
866         * @param appPath application path.
867         * @return coordinator definition.
868         * @throws CoordinatorJobException thrown if the definition could not be read.
869         */
870        protected String readDefinition(String appPath) throws CoordinatorJobException {
871            String user = ParamChecker.notEmpty(conf.get(OozieClient.USER_NAME), OozieClient.USER_NAME);
872            String group = ParamChecker.notEmpty(conf.get(OozieClient.GROUP_NAME), OozieClient.GROUP_NAME);
873            // Configuration confHadoop = CoordUtils.getHadoopConf(conf);
874            try {
875                URI uri = new URI(appPath);
876                LOG.debug("user =" + user + " group =" + group);
877                FileSystem fs = Services.get().get(HadoopAccessorService.class).createFileSystem(user, group, uri,
878                        new Configuration());
879                Path appDefPath = null;
880    
881                // app path could be a directory
882                Path path = new Path(uri.getPath());
883                // check file exists for dataset include file, app xml already checked
884                if (!fs.exists(path)) {
885                    throw new URISyntaxException(path.toString(), "path not existed : " + path.toString());
886                }
887                if (!fs.isFile(path)) {
888                    appDefPath = new Path(path, COORDINATOR_XML_FILE);
889                } else {
890                    appDefPath = path;
891                }
892    
893                Reader reader = new InputStreamReader(fs.open(appDefPath));
894                StringWriter writer = new StringWriter();
895                IOUtils.copyCharStream(reader, writer);
896                return writer.toString();
897            }
898            catch (IOException ex) {
899                LOG.warn("IOException :" + XmlUtils.prettyPrint(conf), ex);
900                throw new CoordinatorJobException(ErrorCode.E1001, ex.getMessage(), ex);
901            }
902            catch (URISyntaxException ex) {
903                LOG.warn("URISyException :" + ex.getMessage());
904                throw new CoordinatorJobException(ErrorCode.E1002, appPath, ex.getMessage(), ex);
905            }
906            catch (HadoopAccessorException ex) {
907                throw new CoordinatorJobException(ex);
908            }
909            catch (Exception ex) {
910                LOG.warn("Exception :", ex);
911                throw new CoordinatorJobException(ErrorCode.E1001, ex.getMessage(), ex);
912            }
913        }
914    
915        /**
916         * Write a coordinator job into database
917         *
918         * @param eJob : XML element of job
919         * @param coordJob : Coordinator job bean
920         * @return Job id
921         * @throws CommandException thrown if unable to save coordinator job to db
922         */
923        private String storeToDB(Element eJob, CoordinatorJobBean coordJob) throws CommandException {
924            String jobId = Services.get().get(UUIDService.class).generateId(ApplicationType.COORDINATOR);
925            coordJob.setId(jobId);
926            coordJob.setAuthToken(this.authToken);
927    
928            if (this.coordName == null) {
929                coordJob.setAppName(eJob.getAttributeValue("name"));
930            }
931    
932            coordJob.setAppPath(conf.get(OozieClient.COORDINATOR_APP_PATH));
933            coordJob.setCreatedTime(new Date());
934            coordJob.setUser(conf.get(OozieClient.USER_NAME));
935            coordJob.setGroup(conf.get(OozieClient.GROUP_NAME));
936            coordJob.setConf(XmlUtils.prettyPrint(conf).toString());
937            coordJob.setJobXml(XmlUtils.prettyPrint(eJob).toString());
938            coordJob.setLastActionNumber(0);
939            coordJob.setLastModifiedTime(new Date());
940    
941            if (!dryrun) {
942                coordJob.setLastModifiedTime(new Date());
943                try {
944                    jpaService.execute(new CoordJobInsertJPAExecutor(coordJob));
945                }
946                catch (JPAExecutorException je) {
947                    throw new CommandException(je);
948                }
949            }
950            return jobId;
951        }
952    
953        /* (non-Javadoc)
954         * @see org.apache.oozie.command.XCommand#getEntityKey()
955         */
956        @Override
957        protected String getEntityKey() {
958            return null;
959        }
960    
961        /* (non-Javadoc)
962         * @see org.apache.oozie.command.XCommand#isLockRequired()
963         */
964        @Override
965        protected boolean isLockRequired() {
966            return false;
967        }
968    
969        /* (non-Javadoc)
970         * @see org.apache.oozie.command.XCommand#loadState()
971         */
972        @Override
973        protected void loadState() throws CommandException {
974            jpaService = Services.get().get(JPAService.class);
975            if (jpaService == null) {
976                throw new CommandException(ErrorCode.E0610);
977            }
978            coordJob = new CoordinatorJobBean();
979            if (this.bundleId != null) {
980                // this coord job is created from bundle
981                coordJob.setBundleId(this.bundleId);
982                // first use bundle id if submit thru bundle
983                logInfo.setParameter(DagXLogInfoService.JOB, this.bundleId);
984                LogUtils.setLogInfo(logInfo);
985            }
986            if (this.coordName != null) {
987                // this coord job is created from bundle
988                coordJob.setAppName(this.coordName);
989            }
990            setJob(coordJob);
991    
992        }
993    
994        /* (non-Javadoc)
995         * @see org.apache.oozie.command.XCommand#verifyPrecondition()
996         */
997        @Override
998        protected void verifyPrecondition() throws CommandException {
999    
1000        }
1001    
1002        /* (non-Javadoc)
1003         * @see org.apache.oozie.command.TransitionXCommand#notifyParent()
1004         */
1005        @Override
1006        public void notifyParent() throws CommandException {
1007            // update bundle action
1008            if (coordJob.getBundleId() != null) {
1009                LOG.debug("Updating bundle record: " + coordJob.getBundleId() + " for coord id: " + coordJob.getId());
1010                BundleStatusUpdateXCommand bundleStatusUpdate = new BundleStatusUpdateXCommand(coordJob, prevStatus);
1011                bundleStatusUpdate.call();
1012            }
1013        }
1014    
1015        /* (non-Javadoc)
1016         * @see org.apache.oozie.command.TransitionXCommand#updateJob()
1017         */
1018        @Override
1019        public void updateJob() throws CommandException {
1020        }
1021    
1022        /* (non-Javadoc)
1023         * @see org.apache.oozie.command.TransitionXCommand#getJob()
1024         */
1025        @Override
1026        public Job getJob() {
1027            return coordJob;
1028        }
1029    }