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.StringReader;
019    import java.util.ArrayList;
020    import java.util.Date;
021    import java.util.List;
022    import java.util.Random;
023    import java.util.concurrent.ConcurrentHashMap;
024    import java.util.concurrent.ConcurrentMap;
025    
026    import org.apache.hadoop.conf.Configuration;
027    import org.apache.hadoop.fs.Path;
028    import org.apache.oozie.CoordinatorActionBean;
029    import org.apache.oozie.ErrorCode;
030    import org.apache.oozie.client.CoordinatorAction;
031    import org.apache.oozie.client.OozieClient;
032    import org.apache.oozie.command.CommandException;
033    import org.apache.oozie.coord.CoordELEvaluator;
034    import org.apache.oozie.coord.CoordELFunctions;
035    import org.apache.oozie.service.CallableQueueService;
036    import org.apache.oozie.service.HadoopAccessorException;
037    import org.apache.oozie.service.HadoopAccessorService;
038    import org.apache.oozie.service.Services;
039    import org.apache.oozie.store.CoordinatorStore;
040    import org.apache.oozie.store.StoreException;
041    import org.apache.oozie.util.DateUtils;
042    import org.apache.oozie.util.ELEvaluator;
043    import org.apache.oozie.util.Instrumentation;
044    import org.apache.oozie.util.ParamChecker;
045    import org.apache.oozie.util.XConfiguration;
046    import org.apache.oozie.util.XLog;
047    import org.apache.oozie.util.XmlUtils;
048    import org.jdom.Element;
049    
050    public class CoordActionInputCheckCommand extends CoordinatorCommand<Void> {
051    
052        private String actionId;
053        private final XLog log = XLog.getLog(getClass());
054        private int COMMAND_REQUEUE_INTERVAL = 60000; // 1 minute
055        private CoordinatorActionBean coordAction = null;
056    
057        private static ConcurrentMap<String, String> ACTION_ID_LOOKUP = new ConcurrentHashMap<String, String>();
058    
059        private static final Random RANDOM = new Random(System.currentTimeMillis());
060    
061        public static boolean queue(CoordActionInputCheckCommand command, long delay) {
062            boolean ret = ACTION_ID_LOOKUP.putIfAbsent(command.actionId,  command.actionId) == null;
063            if (ret) {
064                try {
065                    ret = Services.get().get(CallableQueueService.class).queue(command, delay + RANDOM.nextInt(300));
066                    if (!ret) {
067                        remove(command.actionId);
068                    }
069                }
070                catch (Exception ex) {
071                    remove(command.actionId);
072                }
073            }
074            return ret;
075        }
076    
077        private static void remove(String actionId) {
078           ACTION_ID_LOOKUP.remove(actionId);
079        }
080    
081        private static class ToQueue {
082            private CoordActionInputCheckCommand command;
083            private long delay;
084    
085            private ToQueue(CoordActionInputCheckCommand command, long delay) {
086                this.command = command;
087                this.delay = delay;
088            }
089        }
090    
091        private List<ToQueue> inputChecks = new ArrayList<ToQueue>();
092    
093        private void queueActionInputCheck(CoordActionInputCheckCommand command, long delay) {
094            inputChecks.add(new ToQueue(command, delay));
095        }
096    
097        public CoordActionInputCheckCommand(String actionId) {
098            super("coord_action_input", "coord_action_input", 1, XLog.STD);
099            this.actionId = actionId;
100        }
101    
102        public String toString() {
103            return "AIC-" + actionId;
104        }
105    
106        private void queueActionInputChecks() {
107            for (ToQueue toQueue : inputChecks) {
108                if (!queue(toQueue.command, toQueue.delay)) {
109                    XLog.getLog(CoordActionInputCheckCommand.class).warn("Unable to queue input check command [{0}]",
110                                                                         toQueue.command.actionId);
111                }
112            }
113        }
114    
115        @Override
116        protected Void call(CoordinatorStore store) throws StoreException, CommandException {
117            log.debug("After store.get() for action ID " + actionId + " : " + coordAction.getStatus());
118            // this action should only get processed if current time >
119            // materialization time
120            // otherwise, requeue this action after 30 seconds
121            Date nominalTime = coordAction.getNominalTime();
122            Date currentTime = new Date();
123            if (nominalTime.compareTo(currentTime) > 0) {
124                log.info("[" + actionId
125                        + "]::ActionInputCheck:: nominal Time is newer than current time, so requeue and wait. Current="
126                        + currentTime + ", nominal=" + nominalTime);
127                queueActionInputCheck(new CoordActionInputCheckCommand(coordAction.getId()), Math.max(
128                    (nominalTime.getTime() - currentTime.getTime()), COMMAND_REQUEUE_INTERVAL));
129                // update lastModifiedTime
130                store.updateCoordinatorAction(coordAction);
131                return null;
132            }
133            if (coordAction.getStatus() == CoordinatorActionBean.Status.WAITING) {
134                log.info("[" + actionId + "]::ActionInputCheck:: Action is in WAITING state.");
135                StringBuilder actionXml = new StringBuilder(coordAction.getActionXml());// job.getXml();
136                Instrumentation.Cron cron = new Instrumentation.Cron();
137                try {
138                    Configuration actionConf = new XConfiguration(new StringReader(coordAction.getRunConf()));
139                    cron.start();
140                    StringBuilder existList = new StringBuilder();
141                    StringBuilder nonExistList = new StringBuilder();
142                    StringBuilder nonResolvedList = new StringBuilder();
143                    CoordCommandUtils.getResolvedList(coordAction.getMissingDependencies(), nonExistList,
144                                                                  nonResolvedList);
145    
146                    String[] uriList = nonExistList.toString().split(CoordELFunctions.INSTANCE_SEPARATOR);
147                    if (uriList.length > 0) {
148                        log.info("[" + actionId + "]::ActionInputCheck:: Missing deps:" + uriList[0] + ",  NonResolvedList:"
149                                + nonResolvedList.toString());
150                    } else {
151                        log.info("[" + actionId + "]::ActionInputCheck:: No missing deps,  NonResolvedList:"
152                                + nonResolvedList.toString());
153                    }
154                    Date actualTime = new Date();
155                    boolean status = checkInput(actionXml, existList, nonExistList, actionConf, actualTime);
156                    coordAction.setLastModifiedTime(actualTime);
157                    coordAction.setActionXml(actionXml.toString());
158                    if (nonResolvedList.length() > 0 && status == false) {
159                        nonExistList.append(CoordCommandUtils.RESOLVED_UNRESOLVED_SEPARATOR).append(
160                                nonResolvedList);
161                    }
162                    coordAction.setMissingDependencies(nonExistList.toString());
163                    if (status == true) {
164                        coordAction.setStatus(CoordinatorAction.Status.READY);
165                        // pass jobID to the ReadyCommand
166                        queueCallable(new CoordActionReadyCommand(coordAction.getJobId()), 100);
167                    }
168                    else {
169                        long waitingTime = (actualTime.getTime() - coordAction.getNominalTime().getTime()) / (60 * 1000);
170                        int timeOut = coordAction.getTimeOut();
171                        if ((timeOut >= 0) && (waitingTime > timeOut)) {
172                            queueCallable(new CoordActionTimeOut(coordAction), 100);
173                            coordAction.setStatus(CoordinatorAction.Status.TIMEDOUT);
174                        }
175                        else {
176                            queueActionInputCheck(new CoordActionInputCheckCommand(coordAction.getId()),
177                                                  COMMAND_REQUEUE_INTERVAL);
178                        }
179                    }
180                    store.updateCoordActionMin(coordAction);
181                }
182                catch (Exception e) {
183                    log.warn(actionId + ": Exception occurs: " + e + " STORE is active " + store.isActive(), e);
184                    throw new CommandException(ErrorCode.E1005, e.getMessage(), e);
185                }
186                cron.stop();
187            }
188            else {
189                log.info("[" + actionId + "]::ActionInputCheck:: Ignoring action. Should be in WAITING state, but state="
190                        + coordAction.getStatus());
191            }
192            return null;
193        }
194    
195        protected boolean checkInput(StringBuilder actionXml, StringBuilder existList, StringBuilder nonExistList,
196                                     Configuration conf, Date actualTime) throws Exception {
197            Element eAction = XmlUtils.parseXml(actionXml.toString());
198            boolean allExist = checkResolvedUris(eAction, existList, nonExistList, conf);
199            if (allExist) {
200                log.debug("[" + actionId + "]::ActionInputCheck:: Checking Latest/future");
201                allExist = checkUnresolvedInstances(eAction, conf, actualTime);
202            }
203            if (allExist == true) {
204                materializeDataProperties(eAction, conf);
205                actionXml.replace(0, actionXml.length(), XmlUtils.prettyPrint(eAction).toString());
206            }
207            return allExist;
208        }
209    
210        /**
211         * Materialize data properties defined in <action> tag. it includes dataIn(<DS>) and dataOut(<DS>) it creates a list
212         * of files that will be needed.
213         *
214         * @param eAction
215         * @param conf
216         * @throws Exception
217         * @update modify 'Action' element with appropriate list of files.
218         */
219        private void materializeDataProperties(Element eAction, Configuration conf) throws Exception {
220            ELEvaluator eval = CoordELEvaluator.createDataEvaluator(eAction, conf, actionId);
221            Element configElem = eAction.getChild("action", eAction.getNamespace()).getChild("workflow",
222                                                                                             eAction.getNamespace()).getChild("configuration", eAction.getNamespace());
223            if (configElem != null) {
224                for (Element propElem : (List<Element>) configElem.getChildren("property", configElem.getNamespace())) {
225                    resolveTagContents("value", propElem, eval);
226                }
227            }
228        }
229    
230        private void resolveTagContents(String tagName, Element elem, ELEvaluator eval) throws Exception {
231            if (elem == null) {
232                return;
233            }
234            Element tagElem = elem.getChild(tagName, elem.getNamespace());
235            if (tagElem != null) {
236                String updated = CoordELFunctions.evalAndWrap(eval, tagElem.getText());
237                tagElem.removeContent();
238                tagElem.addContent(updated);
239            }
240            else {
241                log.warn(" Value NOT FOUND " + tagName);
242            }
243        }
244    
245        private boolean checkUnresolvedInstances(Element eAction, Configuration actionConf, Date actualTime)
246                throws Exception {
247            String strAction = XmlUtils.prettyPrint(eAction).toString();
248            Date nominalTime = DateUtils.parseDateUTC(eAction.getAttributeValue("action-nominal-time"));
249            StringBuffer resultedXml = new StringBuffer();
250    
251            boolean ret;
252            Element inputList = eAction.getChild("input-events", eAction.getNamespace());
253            if (inputList != null) {
254                ret = materializeUnresolvedEvent(inputList.getChildren("data-in", eAction.getNamespace()),
255                                                 nominalTime, actualTime, actionConf);
256                if (ret == false) {
257                    resultedXml.append(strAction);
258                    return false;
259                }
260            }
261    
262            // Using latest() or future() in output-event is not intuitive.
263            // We need to make
264            // sure, this assumption is correct.
265            Element outputList = eAction.getChild("output-events", eAction.getNamespace());
266            if (outputList != null) {
267                for (Element dEvent : (List<Element>) outputList.getChildren("data-out", eAction.getNamespace())) {
268                    if (dEvent.getChild("unresolved-instances", dEvent.getNamespace()) != null) {
269                        throw new CommandException(ErrorCode.E1006, "coord:latest()/future()",
270                                " not permitted in output-event ");
271                    }
272                }
273                /*
274                 * ret = materializeUnresolvedEvent( (List<Element>)
275                 * outputList.getChildren("data-out", eAction.getNamespace()),
276                 * actualTime, nominalTime, actionConf); if (ret == false) {
277                 * resultedXml.append(strAction); return false; }
278                 */
279            }
280            return true;
281        }
282    
283        private boolean materializeUnresolvedEvent(List<Element> eDataEvents, Date nominalTime, Date actualTime,
284                                                   Configuration conf) throws Exception {
285            for (Element dEvent : eDataEvents) {
286                if (dEvent.getChild("unresolved-instances", dEvent.getNamespace()) == null) {
287                    continue;
288                }
289                ELEvaluator eval = CoordELEvaluator.createLazyEvaluator(actualTime, nominalTime, dEvent, conf);
290                String uresolvedInstance = dEvent.getChild("unresolved-instances", dEvent.getNamespace()).getTextTrim();
291                String unresolvedList[] = uresolvedInstance.split(CoordELFunctions.INSTANCE_SEPARATOR);
292                StringBuffer resolvedTmp = new StringBuffer();
293                for (int i = 0; i < unresolvedList.length; i++) {
294                    String ret = CoordELFunctions.evalAndWrap(eval, unresolvedList[i]);
295                    Boolean isResolved = (Boolean) eval.getVariable("is_resolved");
296                    if (isResolved == false) {
297                        log.info("[" + actionId + "]::Cannot resolve: " + ret);
298                        return false;
299                    }
300                    if (resolvedTmp.length() > 0) {
301                        resolvedTmp.append(CoordELFunctions.INSTANCE_SEPARATOR);
302                    }
303                    resolvedTmp.append((String) eval.getVariable("resolved_path"));
304                }
305                if (resolvedTmp.length() > 0) {
306                    if (dEvent.getChild("uris", dEvent.getNamespace()) != null) {
307                        resolvedTmp.append(CoordELFunctions.INSTANCE_SEPARATOR).append(
308                                dEvent.getChild("uris", dEvent.getNamespace()).getTextTrim());
309                        dEvent.removeChild("uris", dEvent.getNamespace());
310                    }
311                    Element uriInstance = new Element("uris", dEvent.getNamespace());
312                    uriInstance.addContent(resolvedTmp.toString());
313                    dEvent.getContent().add(1, uriInstance);
314                }
315                dEvent.removeChild("unresolved-instances", dEvent.getNamespace());
316            }
317    
318            return true;
319        }
320    
321        private boolean checkResolvedUris(Element eAction, StringBuilder existList, StringBuilder nonExistList,
322                                          Configuration conf) throws IOException {
323    
324            log.info("[" + actionId + "]::ActionInputCheck:: In checkResolvedUris...");
325            Element inputList = eAction.getChild("input-events", eAction.getNamespace());
326            if (inputList != null) {
327                // List<Element> eDataEvents = inputList.getChildren("data-in",
328                // eAction.getNamespace());
329                // for (Element event : eDataEvents) {
330                // Element uris = event.getChild("uris", event.getNamespace());
331                if (nonExistList.length() > 0) {
332                    checkListOfPaths(existList, nonExistList, conf);
333                }
334                // }
335                return nonExistList.length() == 0;
336            }
337            return true;
338        }
339    
340        private boolean checkListOfPaths(StringBuilder existList, StringBuilder nonExistList, Configuration conf)
341                throws IOException {
342    
343            log.info("[" + actionId + "]::ActionInputCheck:: In checkListOfPaths for: " + nonExistList.toString());
344    
345            String[] uriList = nonExistList.toString().split(CoordELFunctions.INSTANCE_SEPARATOR);
346            nonExistList.delete(0, nonExistList.length());
347            boolean allExists = true;
348            String existSeparator = "", nonExistSeparator = "";
349            for (int i = 0; i < uriList.length; i++) {
350                if (allExists) {
351                    allExists = pathExists(uriList[i], conf);
352                    log.info("[" + actionId + "]::ActionInputCheck:: File:" + uriList[i] + ", Exists? :" + allExists);
353                }
354                if (allExists) {
355                    existList.append(existSeparator).append(uriList[i]);
356                    existSeparator = CoordELFunctions.INSTANCE_SEPARATOR;
357                }
358                else {
359                    nonExistList.append(nonExistSeparator).append(uriList[i]);
360                    nonExistSeparator = CoordELFunctions.INSTANCE_SEPARATOR;
361                }
362            }
363            return allExists;
364        }
365    
366        private boolean pathExists(String sPath, Configuration actionConf) throws IOException {
367            log.debug("checking for the file " + sPath);
368            Path path = new Path(sPath);
369            String user = ParamChecker.notEmpty(actionConf.get(OozieClient.USER_NAME), OozieClient.USER_NAME);
370            String group = ParamChecker.notEmpty(actionConf.get(OozieClient.GROUP_NAME), OozieClient.GROUP_NAME);
371            try {
372                return Services.get().get(HadoopAccessorService.class).
373                        createFileSystem(user, group, path.toUri(), actionConf).exists(path);
374            }
375            catch (HadoopAccessorException e) {
376                throw new IOException(e);
377            }
378        }
379    
380        /**
381         * The function create a list of URIs separated by "," using the instances time stamp and URI-template
382         *
383         * @param event : <data-in> event
384         * @param instances : List of time stamp seprated by ","
385         * @param unresolvedInstances : list of instance with latest/future function
386         * @return : list of URIs separated by ",".
387         * @throws Exception
388         */
389        private String createURIs(Element event, String instances, StringBuilder unresolvedInstances) throws Exception {
390            if (instances == null || instances.length() == 0) {
391                return "";
392            }
393            String[] instanceList = instances.split(CoordELFunctions.INSTANCE_SEPARATOR);
394            StringBuilder uris = new StringBuilder();
395    
396            for (int i = 0; i < instanceList.length; i++) {
397                int funcType = CoordCommandUtils.getFuncType(instanceList[i]);
398                if (funcType == CoordCommandUtils.LATEST || funcType == CoordCommandUtils.FUTURE) {
399                    if (unresolvedInstances.length() > 0) {
400                        unresolvedInstances.append(CoordELFunctions.INSTANCE_SEPARATOR);
401                    }
402                    unresolvedInstances.append(instanceList[i]);
403                    continue;
404                }
405                ELEvaluator eval = CoordELEvaluator.createURIELEvaluator(instanceList[i]);
406                // uris.append(eval.evaluate(event.getChild("dataset",
407                // event.getNamespace()).getChild("uri-template",
408                // event.getNamespace()).getTextTrim(), String.class));
409                if (uris.length() > 0) {
410                    uris.append(CoordELFunctions.INSTANCE_SEPARATOR);
411                }
412                uris.append(CoordELFunctions.evalAndWrap(eval, event.getChild("dataset", event.getNamespace()).getChild(
413                        "uri-template", event.getNamespace()).getTextTrim()));
414            }
415            return uris.toString();
416        }
417    
418        @Override
419        protected Void execute(CoordinatorStore store) throws StoreException, CommandException {
420            log.info("STARTED CoordActionInputCheckCommand for actionid=" + actionId);
421            try {
422                coordAction = store.getEntityManager().find(CoordinatorActionBean.class, actionId);
423                if (coordAction != null) {
424                  setLogInfo(coordAction);
425                  if (lock(coordAction.getJobId())) {
426                      call(store);
427                  }
428                  else {
429                      queueActionInputCheck(new CoordActionInputCheckCommand(actionId), LOCK_FAILURE_REQUEUE_INTERVAL);
430                      log.warn("CoordActionInputCheckCommand lock was not acquired - failed jobId=" + coordAction.getJobId()
431                              + ", actionId=" + actionId + ". Requeing the same.");
432                  }
433                }
434                else {
435                  log.warn("CoordActionInputCheckCommand could not find actionId=" + actionId);
436                }
437            }
438            catch (InterruptedException e) {
439                queueActionInputCheck(new CoordActionInputCheckCommand(actionId), LOCK_FAILURE_REQUEUE_INTERVAL);
440                log.warn("CoordActionInputCheckCommand lock acquiring failed with exception " + e.getMessage() + " for jobId="
441                        + coordAction.getJobId() + ", actionId=" + actionId + " Requeing the same.");
442            }
443            finally {
444                log.info("ENDED CoordActionInputCheckCommand for actionid=" + actionId);
445                remove(actionId);
446                queueActionInputChecks();
447            }
448            return null;
449        }
450    
451        /**
452         * @param args
453         * @throws Exception
454         */
455        public static void main(String[] args) throws Exception {
456            new Services().init();
457            String actionId = "0000000-091221141623042-oozie-dani-C@4";
458            try {
459                new CoordActionInputCheckCommand(actionId).call();
460                Thread.sleep(10000);
461            }
462            finally {
463                new Services().destroy();
464            }
465        }
466    
467    }