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.Set;
030    import java.util.TreeSet;
031    
032    import javax.xml.transform.stream.StreamSource;
033    import javax.xml.validation.Validator;
034    
035    import org.apache.hadoop.conf.Configuration;
036    import org.apache.hadoop.fs.FileSystem;
037    import org.apache.hadoop.fs.Path;
038    import org.apache.oozie.CoordinatorJobBean;
039    import org.apache.oozie.ErrorCode;
040    import org.apache.oozie.client.CoordinatorJob;
041    import org.apache.oozie.client.OozieClient;
042    import org.apache.oozie.client.CoordinatorJob.Execution;
043    import org.apache.oozie.command.CommandException;
044    import org.apache.oozie.coord.CoordELEvaluator;
045    import org.apache.oozie.coord.CoordELFunctions;
046    import org.apache.oozie.coord.CoordUtils;
047    import org.apache.oozie.coord.CoordinatorJobException;
048    import org.apache.oozie.coord.TimeUnit;
049    import org.apache.oozie.service.DagXLogInfoService;
050    import org.apache.oozie.service.HadoopAccessorException;
051    import org.apache.oozie.service.SchemaService;
052    import org.apache.oozie.service.Service;
053    import org.apache.oozie.service.Services;
054    import org.apache.oozie.service.UUIDService;
055    import org.apache.oozie.service.HadoopAccessorService;
056    import org.apache.oozie.service.WorkflowAppService;
057    import org.apache.oozie.service.SchemaService.SchemaName;
058    import org.apache.oozie.service.UUIDService.ApplicationType;
059    import org.apache.oozie.store.CoordinatorStore;
060    import org.apache.oozie.store.StoreException;
061    import org.apache.oozie.util.DateUtils;
062    import org.apache.oozie.util.ELEvaluator;
063    import org.apache.oozie.util.IOUtils;
064    import org.apache.oozie.util.ParamChecker;
065    import org.apache.oozie.util.PropertiesUtils;
066    import org.apache.oozie.util.XConfiguration;
067    import org.apache.oozie.util.XLog;
068    import org.apache.oozie.util.XmlUtils;
069    import org.apache.oozie.workflow.WorkflowException;
070    import org.jdom.Attribute;
071    import org.jdom.Element;
072    import org.jdom.JDOMException;
073    import org.jdom.Namespace;
074    import org.xml.sax.SAXException;
075    
076    /**
077     * This class provides the functionalities to resolve a coordinator job XML and write the job information into a DB
078     * table. <p/> Specifically it performs the following functions: 1. Resolve all the variables or properties using job
079     * configurations. 2. Insert all datasets definition as part of the <data-in> and <data-out> tags. 3. Validate the XML
080     * at runtime.
081     */
082    public class CoordSubmitCommand extends CoordinatorCommand<String> {
083    
084        private Configuration conf;
085        private String authToken;
086        private boolean dryrun;
087    
088        public static final String CONFIG_DEFAULT = "coord-config-default.xml";
089        public static final String COORDINATOR_XML_FILE = "coordinator.xml";
090    
091        private static final Set<String> DISALLOWED_USER_PROPERTIES = new HashSet<String>();
092        private static final Set<String> DISALLOWED_DEFAULT_PROPERTIES = new HashSet<String>();
093        /**
094         * Default timeout for normal jobs, in minutes, after which coordinator input check will timeout
095         */
096        public static final String CONF_DEFAULT_TIMEOUT_NORMAL = Service.CONF_PREFIX + "coord.normal.default.timeout";
097    
098        private XLog log = XLog.getLog(getClass());
099        private ELEvaluator evalFreq = null;
100        private ELEvaluator evalNofuncs = null;
101        private ELEvaluator evalData = null;
102        private ELEvaluator evalInst = null;
103        private ELEvaluator evalSla = null;
104    
105        static {
106            String[] badUserProps = {PropertiesUtils.YEAR, PropertiesUtils.MONTH, PropertiesUtils.DAY,
107                    PropertiesUtils.HOUR, PropertiesUtils.MINUTE, PropertiesUtils.DAYS, PropertiesUtils.HOURS,
108                    PropertiesUtils.MINUTES, PropertiesUtils.KB, PropertiesUtils.MB, PropertiesUtils.GB,
109                    PropertiesUtils.TB, PropertiesUtils.PB, PropertiesUtils.RECORDS, PropertiesUtils.MAP_IN,
110                    PropertiesUtils.MAP_OUT, PropertiesUtils.REDUCE_IN, PropertiesUtils.REDUCE_OUT, PropertiesUtils.GROUPS};
111            PropertiesUtils.createPropertySet(badUserProps, DISALLOWED_USER_PROPERTIES);
112    
113            String[] badDefaultProps = {PropertiesUtils.HADOOP_USER, PropertiesUtils.HADOOP_UGI,
114                    WorkflowAppService.HADOOP_JT_KERBEROS_NAME, WorkflowAppService.HADOOP_NN_KERBEROS_NAME};
115            PropertiesUtils.createPropertySet(badUserProps, DISALLOWED_DEFAULT_PROPERTIES);
116            PropertiesUtils.createPropertySet(badDefaultProps, DISALLOWED_DEFAULT_PROPERTIES);
117        }
118    
119        /**
120         * Constructor to create the Coordinator Submit Command.
121         *
122         * @param conf : Configuration for Coordinator job
123         * @param authToken : To be used for authentication
124         */
125        public CoordSubmitCommand(Configuration conf, String authToken) {
126            super("coord_submit", "coord_submit", 1, XLog.STD);
127            this.conf = ParamChecker.notNull(conf, "conf");
128            this.authToken = ParamChecker.notEmpty(authToken, "authToken");
129        }
130    
131        public CoordSubmitCommand(boolean dryrun, Configuration conf, String authToken) {
132            super("coord_submit", "coord_submit", 1, XLog.STD, dryrun);
133            this.conf = ParamChecker.notNull(conf, "conf");
134            this.authToken = ParamChecker.notEmpty(authToken, "authToken");
135            this.dryrun = dryrun;
136            // TODO Auto-generated constructor stub
137        }
138    
139        /*
140         * (non-Javadoc)
141         * 
142         * @see org.apache.oozie.command.Command#call(org.apache.oozie.store.Store)
143         */
144        @Override
145        protected String call(CoordinatorStore store) throws StoreException, CommandException {
146            String jobId = null;
147            log.info("STARTED Coordinator Submit");
148            incrJobCounter(1);
149            CoordinatorJobBean coordJob = new CoordinatorJobBean();
150            try {
151                XLog.Info.get().setParameter(DagXLogInfoService.TOKEN, conf.get(OozieClient.LOG_TOKEN));
152                mergeDefaultConfig();
153    
154                String appXml = readAndValidateXml();
155                coordJob.setOrigJobXml(appXml);
156                log.debug("jobXml after initial validation " + XmlUtils.prettyPrint(appXml).toString());
157                appXml = XmlUtils.removeComments(appXml);
158                initEvaluators();
159                Element eJob = basicResolveAndIncludeDS(appXml, conf, coordJob);
160                log.debug("jobXml after all validation " + XmlUtils.prettyPrint(eJob).toString());
161    
162                jobId = storeToDB(eJob, store, coordJob);
163                // log JOB info for coordinator jobs
164                setLogInfo(coordJob);
165                log = XLog.getLog(getClass());
166    
167                if (!dryrun) {
168                    // submit a command to materialize jobs for the next 1 hour (3600 secs)
169                    // so we don't wait 10 mins for the Service to run.
170                    queueCallable(new CoordJobMatLookupCommand(jobId, 3600), 100);
171                }
172                else {
173                    Date startTime = coordJob.getStartTime();
174                    long startTimeMilli = startTime.getTime();
175                    long endTimeMilli = startTimeMilli + (3600 * 1000);
176                    Date jobEndTime = coordJob.getEndTime();
177                    Date endTime = new Date(endTimeMilli);
178                    if (endTime.compareTo(jobEndTime) > 0) {
179                        endTime = jobEndTime;
180                    }
181                    jobId = coordJob.getId();
182                    log.info("[" + jobId + "]: Update status to PREMATER");
183                    coordJob.setStatus(CoordinatorJob.Status.PREMATER);
184                    CoordActionMaterializeCommand coordActionMatCom = new CoordActionMaterializeCommand(jobId, startTime,
185                                                                                                        endTime);
186                    Configuration jobConf = null;
187                    try {
188                        jobConf = new XConfiguration(new StringReader(coordJob.getConf()));
189                    }
190                    catch (IOException e1) {
191                        log.warn("Configuration parse error. read from DB :" + coordJob.getConf(), e1);
192                    }
193                    String action = coordActionMatCom.materializeJobs(true, coordJob, jobConf, null);
194                    String output = coordJob.getJobXml() + System.getProperty("line.separator")
195                            + "***actions for instance***" + action;
196                    return output;
197                }
198            }
199            catch (CoordinatorJobException ex) {
200                log.warn("ERROR:  ", ex);
201                throw new CommandException(ex);
202            }
203            catch (IllegalArgumentException iex) {
204                log.warn("ERROR:  ", iex);
205                throw new CommandException(ErrorCode.E1003, iex);
206            }
207            catch (Exception ex) {// TODO
208                log.warn("ERROR:  ", ex);
209                throw new CommandException(ErrorCode.E0803, ex);
210            }
211            log.info("ENDED Coordinator Submit jobId=" + jobId);
212            return jobId;
213        }
214    
215        /**
216         * Read the application XML and validate against coordinator Schema
217         *
218         * @return validated coordinator XML
219         * @throws CoordinatorJobException
220         */
221        private String readAndValidateXml() throws CoordinatorJobException {
222            String appPath = ParamChecker.notEmpty(conf.get(OozieClient.COORDINATOR_APP_PATH),
223                                                   OozieClient.COORDINATOR_APP_PATH);// TODO: COORDINATOR_APP_PATH
224            String coordXml = readDefinition(appPath, COORDINATOR_XML_FILE);
225            validateXml(coordXml);
226            return coordXml;
227        }
228    
229        /**
230         * Validate against Coordinator XSD file
231         *
232         * @param xmlContent : Input coordinator xml
233         * @throws CoordinatorJobException
234         */
235        private void validateXml(String xmlContent) throws CoordinatorJobException {
236            javax.xml.validation.Schema schema = Services.get().get(SchemaService.class).getSchema(SchemaName.COORDINATOR);
237            Validator validator = schema.newValidator();
238            // log.warn("XML " + xmlContent);
239            try {
240                validator.validate(new StreamSource(new StringReader(xmlContent)));
241            }
242            catch (SAXException ex) {
243                log.warn("SAXException :", ex);
244                throw new CoordinatorJobException(ErrorCode.E0701, ex.getMessage(), ex);
245            }
246            catch (IOException ex) {
247                // ex.printStackTrace();
248                log.warn("IOException :", ex);
249                throw new CoordinatorJobException(ErrorCode.E0702, ex.getMessage(), ex);
250            }
251        }
252    
253        /**
254         * Merge default configuration with user-defined configuration.
255         *
256         * @throws CommandException
257         */
258        protected void mergeDefaultConfig() throws CommandException {
259            Path configDefault = new Path(conf.get(OozieClient.COORDINATOR_APP_PATH), CONFIG_DEFAULT);
260            // Configuration fsConfig = new Configuration();
261            // log.warn("CONFIG :" + configDefault.toUri());
262            Configuration fsConfig = CoordUtils.getHadoopConf(conf);
263            FileSystem fs;
264            // TODO: which conf?
265            try {
266                String user = ParamChecker.notEmpty(conf.get(OozieClient.USER_NAME), OozieClient.USER_NAME);
267                String group = ParamChecker.notEmpty(conf.get(OozieClient.GROUP_NAME), OozieClient.GROUP_NAME);
268                fs = Services.get().get(HadoopAccessorService.class).createFileSystem(user, group, configDefault.toUri(),
269                                                                                      new Configuration());
270                if (fs.exists(configDefault)) {
271                    Configuration defaultConf = new XConfiguration(fs.open(configDefault));
272                    PropertiesUtils.checkDisallowedProperties(defaultConf, DISALLOWED_DEFAULT_PROPERTIES);
273                    XConfiguration.injectDefaults(defaultConf, conf);
274                }
275                else {
276                    log.info("configDefault Doesn't exist " + configDefault);
277                }
278                PropertiesUtils.checkDisallowedProperties(conf, DISALLOWED_USER_PROPERTIES);
279            }
280            catch (IOException e) {
281                throw new CommandException(ErrorCode.E0702, e.getMessage() + " : Problem reading default config "
282                        + configDefault, e);
283            }
284            catch (HadoopAccessorException e) {
285                throw new CommandException(e);
286            }
287            log.debug("Merged CONF :" + XmlUtils.prettyPrint(conf).toString());
288        }
289    
290        /**
291         * The method resolve all the variables that are defined in configuration. It also include the data set definition
292         * from dataset file into XML.
293         *
294         * @param appXml : Original job XML
295         * @param conf : Configuration of the job
296         * @param coordJob : Coordinator job bean to be populated.
297         * @return : Resolved and modified job XML element.
298         * @throws Exception
299         */
300        public Element basicResolveAndIncludeDS(String appXml, Configuration conf, CoordinatorJobBean coordJob)
301                throws CoordinatorJobException, Exception {
302            Element basicResolvedApp = resolveInitial(conf, appXml, coordJob);
303            includeDataSets(basicResolvedApp, conf);
304            return basicResolvedApp;
305        }
306    
307        /**
308         * Insert data set into data-in and data-out tags.
309         *
310         * @param eAppXml : coordinator application XML
311         * @param eDatasets : DataSet XML
312         * @return updated application
313         */
314        private void insertDataSet(Element eAppXml, Element eDatasets) {
315            // Adding DS definition in the coordinator XML
316            Element inputList = eAppXml.getChild("input-events", eAppXml.getNamespace());
317            if (inputList != null) {
318                for (Element dataIn : (List<Element>) inputList.getChildren("data-in", eAppXml.getNamespace())) {
319                    Element eDataset = findDataSet(eDatasets, dataIn.getAttributeValue("dataset"));
320                    dataIn.getContent().add(0, eDataset);
321                }
322            }
323            Element outputList = eAppXml.getChild("output-events", eAppXml.getNamespace());
324            if (outputList != null) {
325                for (Element dataOut : (List<Element>) outputList.getChildren("data-out", eAppXml.getNamespace())) {
326                    Element eDataset = findDataSet(eDatasets, dataOut.getAttributeValue("dataset"));
327                    dataOut.getContent().add(0, eDataset);
328                }
329            }
330        }
331    
332        /**
333         * Find a specific dataset from a list of Datasets.
334         *
335         * @param eDatasets : List of data sets
336         * @param name : queried data set name
337         * @return one Dataset element. otherwise throw Exception
338         */
339        private static Element findDataSet(Element eDatasets, String name) {
340            for (Element eDataset : (List<Element>) eDatasets.getChildren("dataset", eDatasets.getNamespace())) {
341                if (eDataset.getAttributeValue("name").equals(name)) {
342                    eDataset = (Element) eDataset.clone();
343                    eDataset.detach();
344                    return eDataset;
345                }
346            }
347            throw new RuntimeException("undefined dataset: " + name);
348        }
349    
350        /**
351         * Initialize all the required EL Evaluators.
352         */
353        protected void initEvaluators() {
354            evalFreq = CoordELEvaluator.createELEvaluatorForGroup(conf, "coord-job-submit-freq");
355            evalNofuncs = CoordELEvaluator.createELEvaluatorForGroup(conf, "coord-job-submit-nofuncs");
356            evalInst = CoordELEvaluator.createELEvaluatorForGroup(conf, "coord-job-submit-instances");
357            evalSla = CoordELEvaluator.createELEvaluatorForGroup(conf, "coord-sla-submit");
358        }
359    
360        /**
361         * Resolve basic entities using job Configuration.
362         *
363         * @param conf :Job configuration
364         * @param appXml : Original job XML
365         * @param coordJob : Coordinator job bean to be populated.
366         * @return Resolved job XML element.
367         * @throws Exception
368         */
369        protected Element resolveInitial(Configuration conf, String appXml, CoordinatorJobBean coordJob)
370                throws CoordinatorJobException, Exception {
371            Element eAppXml = XmlUtils.parseXml(appXml);
372            // job's main attributes
373            // frequency
374            String val = resolveAttribute("frequency", eAppXml, evalFreq);
375            int ival = ParamChecker.checkInteger(val, "frequency");
376            ParamChecker.checkGTZero(ival, "frequency");
377            coordJob.setFrequency(ival);
378            TimeUnit tmp = (evalFreq.getVariable("timeunit") == null) ? TimeUnit.MINUTE : ((TimeUnit) evalFreq
379                    .getVariable("timeunit"));
380            addAnAttribute("freq_timeunit", eAppXml, tmp.toString()); // TODO: Store
381            // TimeUnit
382            coordJob.setTimeUnit(CoordinatorJob.Timeunit.valueOf(tmp.toString()));
383            // End Of Duration
384            tmp = evalFreq.getVariable("endOfDuration") == null ? TimeUnit.NONE : ((TimeUnit) evalFreq
385                    .getVariable("endOfDuration"));
386            addAnAttribute("end_of_duration", eAppXml, tmp.toString());
387            // coordJob.setEndOfDuration(tmp) // TODO: Add new attribute in Job bean
388    
389            // start time
390            val = resolveAttribute("start", eAppXml, evalNofuncs);
391            ParamChecker.checkUTC(val, "start");
392            coordJob.setStartTime(DateUtils.parseDateUTC(val));
393            // end time
394            val = resolveAttribute("end", eAppXml, evalNofuncs);
395            ParamChecker.checkUTC(val, "end");
396            coordJob.setEndTime(DateUtils.parseDateUTC(val));
397            // Time zone
398            val = resolveAttribute("timezone", eAppXml, evalNofuncs);
399            ParamChecker.checkTimeZone(val, "timezone");
400            coordJob.setTimeZone(val);
401    
402            // controls
403            val = resolveTagContents("timeout", eAppXml.getChild("controls", eAppXml.getNamespace()), evalNofuncs);
404            if (val == "") {
405                val = Services.get().getConf().get(CONF_DEFAULT_TIMEOUT_NORMAL);
406            }
407    
408            ival = ParamChecker.checkInteger(val, "timeout");
409            // ParamChecker.checkGEZero(ival, "timeout");
410            coordJob.setTimeout(ival);
411            val = resolveTagContents("concurrency", eAppXml.getChild("controls", eAppXml.getNamespace()), evalNofuncs);
412            if (val == "") {
413                val = "-1";
414            }
415            ival = ParamChecker.checkInteger(val, "concurrency");
416            // ParamChecker.checkGEZero(ival, "concurrency");
417            coordJob.setConcurrency(ival);
418            val = resolveTagContents("execution", eAppXml.getChild("controls", eAppXml.getNamespace()), evalNofuncs);
419            if (val == "") {
420                val = Execution.FIFO.toString();
421            }
422            coordJob.setExecution(Execution.valueOf(val));
423            String[] acceptedVals = {Execution.LIFO.toString(), Execution.FIFO.toString(), Execution.LAST_ONLY.toString()};
424            ParamChecker.isMember(val, acceptedVals, "execution");
425    
426            // datasets
427            resolveTagContents("include", eAppXml.getChild("datasets", eAppXml.getNamespace()), evalNofuncs);
428            // for each data set
429            resolveDataSets(eAppXml);
430            HashMap<String, String> dataNameList = new HashMap<String, String>();
431            resolveIOEvents(eAppXml, dataNameList);
432    
433            resolveTagContents("app-path", eAppXml.getChild("action", eAppXml.getNamespace()).getChild("workflow",
434                                                                                                       eAppXml.getNamespace()), evalNofuncs);
435            // TODO: If action or workflow tag is missing, NullPointerException will
436            // occur
437            Element configElem = eAppXml.getChild("action", eAppXml.getNamespace()).getChild("workflow",
438                                                                                             eAppXml.getNamespace()).getChild("configuration", eAppXml.getNamespace());
439            evalData = CoordELEvaluator.createELEvaluatorForDataEcho(conf, "coord-job-submit-data", dataNameList);
440            if (configElem != null) {
441                for (Element propElem : (List<Element>) configElem.getChildren("property", configElem.getNamespace())) {
442                    resolveTagContents("name", propElem, evalData);
443                    // log.warn("Value :");
444                    // Want to check the data-integrity but don't want to modify the
445                    // XML
446                    // for properties only
447                    Element tmpProp = (Element) propElem.clone();
448                    resolveTagContents("value", tmpProp, evalData);
449                    // val = resolveTagContents("value", propElem, evalData);
450                    // log.warn("Value OK :" + val);
451                }
452            }
453            resolveSLA(eAppXml, coordJob);
454            return eAppXml;
455        }
456    
457        private void resolveSLA(Element eAppXml, CoordinatorJobBean coordJob) throws CommandException {
458            // String prefix = XmlUtils.getNamespacePrefix(eAppXml,
459            // SchemaService.SLA_NAME_SPACE_URI);
460            Element eSla = eAppXml.getChild("action", eAppXml.getNamespace()).getChild("info",
461                                                                                       Namespace.getNamespace(SchemaService.SLA_NAME_SPACE_URI));
462    
463            if (eSla != null) {
464                String slaXml = XmlUtils.prettyPrint(eSla).toString();
465                try {
466                    // EL evaluation
467                    slaXml = evalSla.evaluate(slaXml, String.class);
468                    // Validate against semantic SXD
469                    XmlUtils.validateData(slaXml, SchemaName.SLA_ORIGINAL);
470                }
471                catch (Exception e) {
472                    throw new CommandException(ErrorCode.E1004, "Validation ERROR :" + e.getMessage(), e);
473                }
474            }
475        }
476    
477        /**
478         * Resolve input-events/data-in and output-events/data-out tags.
479         *
480         * @param eJob : Job element
481         * @throws CoordinatorJobException
482         */
483        private void resolveIOEvents(Element eJobOrg, HashMap<String, String> dataNameList) throws CoordinatorJobException {
484            // Resolving input-events/data-in
485            // Clone the job and don't update anything in the original
486            Element eJob = (Element) eJobOrg.clone();
487            Element inputList = eJob.getChild("input-events", eJob.getNamespace());
488            if (inputList != null) {
489                TreeSet<String> eventNameSet = new TreeSet<String>();
490                for (Element dataIn : (List<Element>) inputList.getChildren("data-in", eJob.getNamespace())) {
491                    String dataInName = dataIn.getAttributeValue("name");
492                    dataNameList.put(dataInName, "data-in");
493                    // check whether there is any duplicate data-in name
494                    if (eventNameSet.contains(dataInName)) {
495                        throw new RuntimeException("Duplicate dataIn name " + dataInName);
496                    }
497                    else {
498                        eventNameSet.add(dataInName);
499                    }
500                    resolveTagContents("instance", dataIn, evalInst);
501                    resolveTagContents("start-instance", dataIn, evalInst);
502                    resolveTagContents("end-instance", dataIn, evalInst);
503                }
504            }
505            // Resolving output-events/data-out
506            Element outputList = eJob.getChild("output-events", eJob.getNamespace());
507            if (outputList != null) {
508                TreeSet<String> eventNameSet = new TreeSet<String>();
509                for (Element dataOut : (List<Element>) outputList.getChildren("data-out", eJob.getNamespace())) {
510                    String dataOutName = dataOut.getAttributeValue("name");
511                    dataNameList.put(dataOutName, "data-out");
512                    // check whether there is any duplicate data-out name
513                    if (eventNameSet.contains(dataOutName)) {
514                        throw new RuntimeException("Duplicate dataIn name " + dataOutName);
515                    }
516                    else {
517                        eventNameSet.add(dataOutName);
518                    }
519                    resolveTagContents("instance", dataOut, evalInst);
520                }
521            }
522    
523        }
524    
525        /**
526         * Add an attribute into XML element.
527         *
528         * @param attrName :attribute name
529         * @param elem : Element to add attribute
530         * @param value :Value of attribute
531         */
532        private void addAnAttribute(String attrName, Element elem, String value) {
533            elem.setAttribute(attrName, value);
534        }
535    
536        /**
537         * Resolve Data set using job configuration.
538         *
539         * @param eAppXml : Job Element XML
540         * @throws Exception
541         */
542        private void resolveDataSets(Element eAppXml) throws Exception {
543            Element datasetList = eAppXml.getChild("datasets", eAppXml.getNamespace());
544            if (datasetList != null) {
545    
546                List<Element> dsElems = datasetList.getChildren("dataset", eAppXml.getNamespace());
547                resolveDataSets(dsElems);
548                resolveTagContents("app-path", eAppXml.getChild("action", eAppXml.getNamespace()).getChild("workflow",
549                                                                                                           eAppXml.getNamespace()), evalNofuncs);
550            }
551        }
552    
553        /**
554         * Resolve Data set using job configuration.
555         *
556         * @param dsElems : Data set XML element.
557         * @throws CoordinatorJobException
558         * @throws Exception
559         */
560        private void resolveDataSets(List<Element> dsElems) throws CoordinatorJobException /*
561                                                                                            * throws
562                                                                                            * Exception
563                                                                                            */ {
564            for (Element dsElem : dsElems) {
565                // Setting up default TimeUnit and EndOFDuraion
566                evalFreq.setVariable("timeunit", TimeUnit.MINUTE);
567                evalFreq.setVariable("endOfDuration", TimeUnit.NONE);
568    
569                String val = resolveAttribute("frequency", dsElem, evalFreq);
570                int ival = ParamChecker.checkInteger(val, "frequency");
571                ParamChecker.checkGTZero(ival, "frequency");
572                addAnAttribute("freq_timeunit", dsElem, evalFreq.getVariable("timeunit") == null ? TimeUnit.MINUTE
573                        .toString() : ((TimeUnit) evalFreq.getVariable("timeunit")).toString());
574                addAnAttribute("end_of_duration", dsElem, evalFreq.getVariable("endOfDuration") == null ? TimeUnit.NONE
575                        .toString() : ((TimeUnit) evalFreq.getVariable("endOfDuration")).toString());
576                val = resolveAttribute("initial-instance", dsElem, evalNofuncs);
577                ParamChecker.checkUTC(val, "initial-instance");
578                val = resolveAttribute("timezone", dsElem, evalNofuncs);
579                ParamChecker.checkTimeZone(val, "timezone");
580                resolveTagContents("uri-template", dsElem, evalNofuncs);
581                resolveTagContents("done-flag", dsElem, evalNofuncs);
582            }
583        }
584    
585        /**
586         * Resolve the content of a tag.
587         *
588         * @param tagName : Tag name of job XML i.e. <timeout> 10 </timeout>
589         * @param elem : Element where the tag exists.
590         * @param eval :
591         * @return Resolved tag content.
592         * @throws CoordinatorJobException
593         */
594        private String resolveTagContents(String tagName, Element elem, ELEvaluator eval) throws CoordinatorJobException {
595            String ret = "";
596            if (elem != null) {
597                for (Element tagElem : (List<Element>) elem.getChildren(tagName, elem.getNamespace())) {
598                    if (tagElem != null) {
599                        String updated;
600                        try {
601                            updated = CoordELFunctions.evalAndWrap(eval, tagElem.getText().trim());
602    
603                        }
604                        catch (Exception e) {
605                            // e.printStackTrace();
606                            throw new CoordinatorJobException(ErrorCode.E1004, e.getMessage(), e);
607                        }
608                        tagElem.removeContent();
609                        tagElem.addContent(updated);
610                        ret += updated;
611                    }
612                    /*
613                     * else { //TODO: unlike event }
614                     */
615                }
616            }
617            return ret;
618        }
619    
620        /**
621         * Resolve an attribute value.
622         *
623         * @param attrName : Attribute name.
624         * @param elem : XML Element where attribute is defiend
625         * @param eval : ELEvaluator used to resolve
626         * @return Resolved attribute value
627         * @throws CoordinatorJobException
628         */
629        private String resolveAttribute(String attrName, Element elem, ELEvaluator eval) throws CoordinatorJobException {
630            Attribute attr = elem.getAttribute(attrName);
631            String val = null;
632            if (attr != null) {
633                try {
634                    val = CoordELFunctions.evalAndWrap(eval, attr.getValue().trim());
635    
636                }
637                catch (Exception e) {
638                    // e.printStackTrace();
639                    throw new CoordinatorJobException(ErrorCode.E1004, e.getMessage(), e);
640                }
641                attr.setValue(val);
642            }
643            return val;
644        }
645    
646        /**
647         * Include referred Datasets into XML.
648         *
649         * @param resolvedXml : Job XML element.
650         * @param conf : Job configuration
651         * @throws CoordinatorJobException
652         */
653        protected void includeDataSets(Element resolvedXml, Configuration conf) throws CoordinatorJobException
654            /* throws Exception */ {
655            Element datasets = resolvedXml.getChild("datasets", resolvedXml.getNamespace());
656            Element allDataSets = new Element("all_datasets", resolvedXml.getNamespace());
657            List<String> dsList = new ArrayList<String>();
658            if (datasets != null) {
659                for (Element includeElem : (List<Element>) datasets.getChildren("include", datasets.getNamespace())) {
660                    String incDSFile = includeElem.getTextTrim();
661                    // log.warn(" incDSFile " + incDSFile);
662                    includeOneDSFile(incDSFile, dsList, allDataSets, datasets.getNamespace());
663                }
664                for (Element e : (List<Element>) datasets.getChildren("dataset", datasets.getNamespace())) {
665                    String dsName = (String) e.getAttributeValue("name");
666                    if (dsList.contains(dsName)) {// Override with this DS
667                        // Remove old DS
668                        removeDataSet(allDataSets, dsName);
669                        // throw new RuntimeException("Duplicate Dataset " +
670                        // dsName);
671                    }
672                    else {
673                        dsList.add(dsName);
674                    }
675                    allDataSets.addContent((Element) e.clone());
676                }
677            }
678            insertDataSet(resolvedXml, allDataSets);
679            resolvedXml.removeChild("datasets", resolvedXml.getNamespace());
680        }
681    
682        /**
683         * Include One Dataset file.
684         *
685         * @param incDSFile : Include data set filename.
686         * @param dsList :List of dataset names to verify the duplicate.
687         * @param allDataSets : Element that includes all dataset definitions.
688         * @param dsNameSpace : Data set name space
689         * @throws CoordinatorJobException
690         * @throws Exception
691         */
692        private void includeOneDSFile(String incDSFile, List<String> dsList, Element allDataSets, Namespace dsNameSpace)
693                throws CoordinatorJobException {
694            Element tmpDataSets = null;
695            try {
696                String dsXml = readDefinition(incDSFile, "");
697                log.debug("DSFILE :" + incDSFile + "\n" + dsXml);
698                tmpDataSets = XmlUtils.parseXml(dsXml);
699            }
700            /*
701             * catch (IOException iex) {XLog.getLog(getClass()).warn(
702             * "Error reading included dataset file [{0}].  Message [{1}]",
703             * incDSFile, iex.getMessage()); throw new
704             * CommandException(ErrorCode.E0803, iex.getMessage()); }
705             */
706            catch (JDOMException e) {
707                log.warn("Error parsing included dataset [{0}].  Message [{1}]", incDSFile, e.getMessage());
708                throw new CoordinatorJobException(ErrorCode.E0700, e.getMessage());
709            }
710            resolveDataSets((List<Element>) tmpDataSets.getChildren("dataset"));
711            for (Element e : (List<Element>) tmpDataSets.getChildren("dataset")) {
712                String dsName = (String) e.getAttributeValue("name");
713                if (dsList.contains(dsName)) {
714                    throw new RuntimeException("Duplicate Dataset " + dsName);
715                }
716                dsList.add(dsName);
717                Element tmp = (Element) e.clone();
718                // TODO: Don't like to over-write the external/include DS's
719                // namespace
720                tmp.setNamespace(dsNameSpace);// TODO:
721                tmp.getChild("uri-template").setNamespace(dsNameSpace);
722                if (e.getChild("done-flag") != null) {
723                    tmp.getChild("done-flag").setNamespace(dsNameSpace);
724                }
725                allDataSets.addContent(tmp);
726            }
727            // nested include
728            for (Element includeElem : (List<Element>) tmpDataSets.getChildren("include", tmpDataSets.getNamespace())) {
729                String incFile = includeElem.getTextTrim();
730                // log.warn("incDSFile "+ incDSFile);
731                includeOneDSFile(incFile, dsList, allDataSets, dsNameSpace);
732            }
733        }
734    
735        /**
736         * Remove a dataset from a list of dataset.
737         *
738         * @param eDatasets : List of dataset
739         * @param name : Dataset name to be removed.
740         */
741        private static void removeDataSet(Element eDatasets, String name) {
742            for (Element eDataset : (List<Element>) eDatasets.getChildren("dataset", eDatasets.getNamespace())) {
743                if (eDataset.getAttributeValue("name").equals(name)) {
744                    eDataset.detach();
745                }
746            }
747            throw new RuntimeException("undefined dataset: " + name);
748        }
749    
750        /**
751         * Read workflow definition.
752         *
753         * @param appPath application path.
754         * @param user user name.
755         * @param group group name.
756         * @param autToken authentication token.
757         * @return workflow definition.
758         * @throws WorkflowException thrown if the definition could not be read.
759         */
760        protected String readDefinition(String appPath, String fileName) throws CoordinatorJobException {// TODO:
761            String user = ParamChecker.notEmpty(conf.get(OozieClient.USER_NAME), OozieClient.USER_NAME);
762            String group = ParamChecker.notEmpty(conf.get(OozieClient.GROUP_NAME), OozieClient.GROUP_NAME);
763            Configuration confHadoop = CoordUtils.getHadoopConf(conf);
764            try {
765                URI uri = new URI(appPath);
766                log.debug("user =" + user + " group =" + group);
767                FileSystem fs = Services.get().get(HadoopAccessorService.class).createFileSystem(user, group, uri,
768                                                                                                 new Configuration());
769                Path p;
770                if (fileName == null || fileName.length() == 0) {
771                    p = new Path(uri.getPath());
772                }
773                else {
774                    p = new Path(uri.getPath(), fileName);
775                }
776                // Reader reader = new InputStreamReader(fs.open(new Path(uri
777                // .getPath(), fileName)));
778                Reader reader = new InputStreamReader(fs.open(p));// TODO
779                StringWriter writer = new StringWriter();
780                IOUtils.copyCharStream(reader, writer);
781                return writer.toString();
782            }
783            catch (IOException ex) {
784                log.warn("IOException :" + XmlUtils.prettyPrint(confHadoop), ex);
785                throw new CoordinatorJobException(ErrorCode.E1001, ex.getMessage(), ex); // TODO:
786            }
787            catch (URISyntaxException ex) {
788                log.warn("URISyException :" + ex.getMessage());
789                throw new CoordinatorJobException(ErrorCode.E1002, appPath, ex.getMessage(), ex);// TODO:
790            }
791            catch (HadoopAccessorException ex) {
792                throw new CoordinatorJobException(ex);
793            }
794            catch (Exception ex) {
795                log.warn("Exception :", ex);
796                throw new CoordinatorJobException(ErrorCode.E1001, ex.getMessage(), ex);// TODO:
797            }
798        }
799    
800        /**
801         * Write a Coordinator Job into database
802         *
803         * @param eJob : XML element of job
804         * @param store : Coordinator Store to write.
805         * @param coordJob : Coordinator job bean
806         * @return Job if.
807         * @throws StoreException
808         */
809        private String storeToDB(Element eJob, CoordinatorStore store, CoordinatorJobBean coordJob) throws StoreException {
810            String jobId = Services.get().get(UUIDService.class).generateId(ApplicationType.COORDINATOR);
811            coordJob.setId(jobId);
812            coordJob.setAuthToken(this.authToken);
813            coordJob.setAppName(eJob.getAttributeValue("name"));
814            coordJob.setAppPath(conf.get(OozieClient.COORDINATOR_APP_PATH));
815            coordJob.setStatus(CoordinatorJob.Status.PREP);
816            coordJob.setCreatedTime(new Date()); // TODO: Do we need that?
817            coordJob.setUser(conf.get(OozieClient.USER_NAME));
818            coordJob.setGroup(conf.get(OozieClient.GROUP_NAME));
819            coordJob.setConf(XmlUtils.prettyPrint(conf).toString());
820            coordJob.setJobXml(XmlUtils.prettyPrint(eJob).toString());
821            coordJob.setLastActionNumber(0);
822            coordJob.setLastModifiedTime(new Date());
823    
824            if (!dryrun) {
825                store.insertCoordinatorJob(coordJob);
826            }
827            return jobId;
828        }
829    
830        /**
831         * For unit-testing only. Will ultimately go away
832         *
833         * @param args
834         * @throws Exception
835         * @throws JDOMException
836         */
837        public static void main(String[] args) throws Exception {
838            // TODO Auto-generated method stub
839            // Configuration conf = new XConfiguration(IOUtils.getResourceAsReader(
840            // "org/apache/oozie/coord/conf.xml", -1));
841    
842            Configuration conf = new XConfiguration();
843    
844            // base case
845            // conf.set(OozieClient.COORDINATOR_APP_PATH,
846            // "file:///Users/danielwo/oozie/workflows/coord/test1/");
847    
848            // no input datasets
849            // conf.set(OozieClient.COORDINATOR_APP_PATH,
850            // "file:///Users/danielwo/oozie/workflows/coord/coord_noinput/");
851            // conf.set(OozieClient.COORDINATOR_APP_PATH,
852            // "file:///Users/danielwo/oozie/workflows/coord/coord_use_apppath/");
853    
854            // only 1 instance
855            // conf.set(OozieClient.COORDINATOR_APP_PATH,
856            // "file:///Users/danielwo/oozie/workflows/coord/coord_oneinstance/");
857    
858            // no local props in xml
859            // conf.set(OozieClient.COORDINATOR_APP_PATH,
860            // "file:///Users/danielwo/oozie/workflows/coord/coord_noprops/");
861    
862            conf.set(OozieClient.COORDINATOR_APP_PATH,
863                     "file:///homes/test/workspace/sandbox_krishna/oozie-main/core/src/main/java/org/apache/oozie/coord/");
864            conf.set(OozieClient.USER_NAME, "test");
865            // conf.set(OozieClient.USER_NAME, "danielwo");
866            conf.set(OozieClient.GROUP_NAME, "other");
867            // System.out.println("appXml :"+ appXml + "\n conf :"+ conf);
868            new Services().init();
869            try {
870                CoordSubmitCommand sc = new CoordSubmitCommand(conf, "TESTING");
871                String jobId = sc.call();
872                System.out.println("Job Id " + jobId);
873                Thread.sleep(80000);
874            }
875            finally {
876                Services.get().destroy();
877            }
878        }
879    }