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;
016    
017    import java.util.ArrayList;
018    import java.util.List;
019    
020    import org.apache.oozie.CoordinatorActionBean;
021    import org.apache.oozie.CoordinatorJobBean;
022    import org.apache.oozie.ErrorCode;
023    import org.apache.oozie.FaultInjection;
024    import org.apache.oozie.WorkflowActionBean;
025    import org.apache.oozie.WorkflowJobBean;
026    import org.apache.oozie.XException;
027    import org.apache.oozie.service.CallableQueueService;
028    import org.apache.oozie.service.DagXLogInfoService;
029    import org.apache.oozie.service.InstrumentationService;
030    import org.apache.oozie.service.MemoryLocksService;
031    import org.apache.oozie.service.Services;
032    import org.apache.oozie.service.StoreService;
033    import org.apache.oozie.service.XLogService;
034    import org.apache.oozie.store.Store;
035    import org.apache.oozie.store.StoreException;
036    import org.apache.oozie.store.WorkflowStore;
037    import org.apache.oozie.util.Instrumentation;
038    import org.apache.oozie.util.ParamChecker;
039    import org.apache.oozie.util.XCallable;
040    import org.apache.oozie.util.XLog;
041    import org.apache.oozie.util.MemoryLocks.LockToken;
042    
043    /**
044     * Base class for all synchronous and asynchronous DagEngine commands.
045     */
046    public abstract class Command<T, S extends Store> implements XCallable<T> {
047        /**
048         * The instrumentation group used for Commands.
049         */
050        private static final String INSTRUMENTATION_GROUP = "commands";
051    
052        private final long createdTime;
053    
054        /**
055         * The instrumentation group used for Jobs.
056         */
057        private static final String INSTRUMENTATION_JOB_GROUP = "jobs";
058    
059        private static final long LOCK_TIMEOUT = 1000;
060        protected static final long LOCK_FAILURE_REQUEUE_INTERVAL = 30000;
061    
062        protected Instrumentation instrumentation;
063        private List<XCallable<Void>> callables;
064        private List<XCallable<Void>> delayedCallables;
065        private long delay = 0;
066        private List<XCallable<Void>> exceptionCallables;
067        private String name;
068        private int priority;
069        private int logMask;
070        private boolean withStore;
071        protected boolean dryrun = false;
072        protected String type;
073        private ArrayList<LockToken> locks = null;
074    
075        /**
076         * This variable is package private for testing purposes only.
077         */
078        XLog.Info logInfo;
079    
080        /**
081         * Create a command that uses a {@link WorkflowStore} instance. <p/> The current {@link XLog.Info} values are
082         * captured for execution.
083         *
084         * @param name command name.
085         * @param type command type.
086         * @param priority priority of the command, used when queuing for asynchronous execution.
087         * @param logMask log mask for the command logging calls.
088         */
089        public Command(String name, String type, int priority, int logMask) {
090            this(name, type, priority, logMask, true);
091        }
092    
093        /**
094         * Create a command. <p/> The current {@link XLog.Info} values are captured for execution.
095         *
096         * @param name command name.
097         * @param type command type.
098         * @param priority priority of the command, used when queuing for asynchronous execution.
099         * @param logMask log mask for the command logging calls.
100         * @param withStore indicates if the command needs a {@link org.apache.oozie.store.WorkflowStore} instance or not.
101         */
102        public Command(String name, String type, int priority, int logMask, boolean withStore) {
103            this.name = ParamChecker.notEmpty(name, "name");
104            this.type = ParamChecker.notEmpty(type, "type");
105            this.priority = priority;
106            this.withStore = withStore;
107            this.logMask = logMask;
108            instrumentation = Services.get().get(InstrumentationService.class).get();
109            logInfo = new XLog.Info(XLog.Info.get());
110            createdTime = System.currentTimeMillis();
111            locks = new ArrayList<LockToken>();
112        }
113    
114        /**
115         * Create a command. <p/> The current {@link XLog.Info} values are captured for execution.
116         *
117         * @param name command name.
118         * @param type command type.
119         * @param priority priority of the command, used when queuing for asynchronous execution.
120         * @param logMask log mask for the command logging calls.
121         * @param withStore indicates if the command needs a {@link org.apache.oozie.store.WorkflowStore} instance or not.
122         * @param dryrun indicates if dryrun option is enabled. if enabled coordinator will show a diagnostic output without
123         * really submitting the job
124         */
125        public Command(String name, String type, int priority, int logMask, boolean withStore, boolean dryrun) {
126            this(name, type, priority, logMask, withStore);
127            this.dryrun = dryrun;
128        }
129    
130        /**
131         * Return the name of the command.
132         *
133         * @return the name of the command.
134         */
135        @Override
136        public String getName() {
137            return name;
138        }
139    
140        /**
141         * Return the callable type. <p/> The callable type is used for concurrency throttling in the {@link
142         * org.apache.oozie.service.CallableQueueService}.
143         *
144         * @return the callable type.
145         */
146        @Override
147        public String getType() {
148            return type;
149        }
150    
151        /**
152         * Return the priority of the command.
153         *
154         * @return the priority of the command.
155         */
156        @Override
157        public int getPriority() {
158            return priority;
159        }
160    
161        /**
162         * Returns the createdTime of the callable in milliseconds
163         *
164         * @return the callable createdTime
165         */
166        @Override
167        public long getCreatedTime() {
168            return createdTime;
169        }
170    
171        /**
172         * Execute the command {@link #call(WorkflowStore)} setting all the necessary context. <p/> The {@link XLog.Info} is
173         * set to the values at instance creation time. <p/> The command execution is logged and instrumented. <p/> If a
174         * {@link WorkflowStore} is used, a fresh instance will be passed and it will be commited after the {@link
175         * #call(WorkflowStore)} execution. It will be closed without committing if an exception is thrown. <p/> Commands
176         * queued via the DagCommand queue methods are queued for execution after the workflow store has been committed.
177         * <p/> If an exception happends the queued commands will not be effectively queued for execution. Instead, the the
178         * commands queued for exception will be effectively queued fro execution..
179         *
180         * @throws CommandException thrown if the command could not be executed successfully, the workflow store is closed
181         * without committing, thus doing a rollback.
182         */
183        @SuppressWarnings({"ThrowFromFinallyBlock", "unchecked"})
184        public final T call() throws CommandException {
185            XLog.Info.get().setParameters(logInfo);
186            XLog log = XLog.getLog(getClass());
187            log.trace(logMask, "Start");
188            Instrumentation.Cron cron = new Instrumentation.Cron();
189            cron.start();
190            callables = new ArrayList<XCallable<Void>>();
191            delayedCallables = new ArrayList<XCallable<Void>>();
192            exceptionCallables = new ArrayList<XCallable<Void>>();
193            delay = 0;
194            S store = null;
195            boolean exception = false;
196    
197            try {
198                if (withStore) {
199                    store = (S) Services.get().get(StoreService.class).getStore(getStoreClass());
200                    store.beginTrx();
201                }
202                T result = execute(store);
203                /*
204                 *
205                 * if (store != null && log != null) { log.info(XLog.STD,
206                 * "connection log from store Flush Mode {0} ",
207                 * store.getFlushMode()); }
208                 */
209                if (withStore) {
210                    if (store == null) {
211                        throw new IllegalStateException("WorkflowStore should not be null");
212                    }
213                    if (FaultInjection.isActive("org.apache.oozie.command.SkipCommitFaultInjection")) {
214                        throw new RuntimeException("Skipping Commit for Failover Testing");
215                    }
216                    store.commitTrx();
217                }
218    
219                // TODO figure out the reject due to concurrency problems and remove
220                // the delayed queuing for callables.
221                boolean ret = Services.get().get(CallableQueueService.class).queueSerial(callables, 10);
222                if (ret == false) {
223                    logQueueCallableFalse(callables);
224                }
225    
226                ret = Services.get().get(CallableQueueService.class).queueSerial(delayedCallables, delay);
227                if (ret == false) {
228                    logQueueCallableFalse(delayedCallables);
229                }
230    
231                return result;
232            }
233            catch (XException ex) {
234                log.error(logMask | XLog.OPS, "XException, {0}", ex);
235                if (store != null) {
236                    log.info(XLog.STD, "XException - connection logs from store {0}, {1}", store.getConnection(), store
237                            .isClosed());
238                }
239                exception = true;
240                if (store != null && store.isActive()) {
241                    try {
242                        store.rollbackTrx();
243                    }
244                    catch (RuntimeException rex) {
245                        log.warn(logMask | XLog.OPS, "openjpa error, {0}, {1}", name, rex.getMessage(), rex);
246                    }
247                }
248    
249                // TODO figure out the reject due to concurrency problems and remove
250                // the delayed queuing for callables.
251                boolean ret = Services.get().get(CallableQueueService.class).queueSerial(exceptionCallables, 10);
252                if (ret == false) {
253                    logQueueCallableFalse(exceptionCallables);
254                }
255                if (ex instanceof CommandException) {
256                    throw (CommandException) ex;
257                }
258                else {
259                    throw new CommandException(ex);
260                }
261            }
262            catch (Exception ex) {
263                log.error(logMask | XLog.OPS, "Exception, {0}", ex);
264                exception = true;
265                if (store != null && store.isActive()) {
266                    try {
267                        store.rollbackTrx();
268                    }
269                    catch (RuntimeException rex) {
270                        log.warn(logMask | XLog.OPS, "openjpa error, {0}, {1}", name, rex.getMessage(), rex);
271                    }
272                }
273                throw new CommandException(ErrorCode.E0607, ex);
274            }
275            catch (Error er) {
276                log.error(logMask | XLog.OPS, "Error, {0}", er);
277                exception = true;
278                if (store != null && store.isActive()) {
279                    try {
280                        store.rollbackTrx();
281                    }
282                    catch (RuntimeException rex) {
283                        log.warn(logMask | XLog.OPS, "openjpa error, {0}, {1}", name, rex.getMessage(), rex);
284                    }
285                }
286                throw er;
287            }
288            finally {
289                FaultInjection.deactivate("org.apache.oozie.command.SkipCommitFaultInjection");
290                cron.stop();
291                instrumentation.addCron(INSTRUMENTATION_GROUP, name, cron);
292                incrCommandCounter(1);
293                log.trace(logMask, "End");
294                if (locks != null) {
295                    for (LockToken lock : locks) {
296                        lock.release();
297                    }
298                    locks.clear();
299                }
300                if (store != null) {
301                    if (!store.isActive()) {
302                        try {
303                            store.closeTrx();
304                        }
305                        catch (RuntimeException rex) {
306                            if (exception) {
307                                log.warn(logMask | XLog.OPS, "openjpa error, {0}, {1}", name, rex.getMessage(), rex);
308                            }
309                            else {
310                                throw rex;
311                            }
312                        }
313                    }
314                    else {
315                        log.warn(logMask | XLog.OPS, "transaction is not committed or rolled back before closing entitymanager.");
316                    }
317                }
318            }
319        }
320    
321        /**
322         * Queue a callable for execution after the current callable call invocation completes and the {@link WorkflowStore}
323         * transaction commits. <p/> All queued callables, regardless of the number of queue invocations, are queued for a
324         * single serial execution. <p/> If the call invocation throws an exception all queued callables are discarded, they
325         * are not queued for execution.
326         *
327         * @param callable callable to queue for execution.
328         */
329        protected void queueCallable(XCallable<Void> callable) {
330            callables.add(callable);
331        }
332    
333        /**
334         * Queue a list of callables for execution after the current callable call invocation completes and the {@link
335         * WorkflowStore} transaction commits. <p/> All queued callables, regardless of the number of queue invocations, are
336         * queued for a single serial execution. <p/> If the call invocation throws an exception all queued callables are
337         * discarded, they are not queued for execution.
338         *
339         * @param callables list of callables to queue for execution.
340         */
341        protected void queueCallable(List<? extends XCallable<Void>> callables) {
342            this.callables.addAll(callables);
343        }
344    
345        /**
346         * Queue a callable for delayed execution after the current callable call invocation completes and the {@link
347         * WorkflowStore} transaction commits. <p/> All queued delayed callables, regardless of the number of delay queue
348         * invocations, are queued for a single serial delayed execution with the highest delay of all queued callables.
349         * <p/> If the call invocation throws an exception all queued callables are discarded, they are not queued for
350         * execution.
351         *
352         * @param callable callable to queue for delayed execution.
353         * @param delay the queue delay in milliseconds
354         */
355        protected void queueCallable(XCallable<Void> callable, long delay) {
356            this.delayedCallables.add(callable);
357            this.delay = Math.max(this.delay, delay);
358        }
359    
360        /**
361         * Queue a callable for execution only in the event of an exception being thrown during the call invocation. <p/> If
362         * an exception does not happen, all the callables queued by this method are discarded, they are not queued for
363         * execution. <p/> All queued callables, regardless of the number of queue invocations, are queued for a single
364         * serial execution.
365         *
366         * @param callable callable to queue for execution in the case of an exception.
367         */
368        protected void queueCallableForException(XCallable<Void> callable) {
369            exceptionCallables.add(callable);
370        }
371    
372        /**
373         * Logging the info if failed to queue the callables.
374         *
375         * @param callables
376         */
377        protected void logQueueCallableFalse(List<? extends XCallable<Void>> callables) {
378            StringBuilder sb = new StringBuilder(
379                    "Unable to queue the callables, delayedQueue is full or system is in SAFEMODE - failed to queue:[");
380            int size = callables.size();
381            for (int i = 0; i < size; i++) {
382                XCallable<Void> callable = callables.get(i);
383                sb.append(callable.getName());
384                if (i < size - 1) {
385                    sb.append(", ");
386                }
387                else {
388                    sb.append("]");
389                }
390            }
391            XLog.getLog(getClass()).warn(sb.toString());
392        }
393    
394        /**
395         * DagCallable subclasses must implement this method to perform their task. <p/> The workflow store works in
396         * transactional mode. The transaction is committed only if this method ends successfully. Otherwise the transaction
397         * is rolledback.
398         *
399         * @param store the workflow store instance for the callable, <code>null</code> if the callable does not use a
400         * store.
401         * @return the return value of the callable.
402         * @throws StoreException thrown if the workflow store could not perform an operation.
403         * @throws CommandException thrown if the command could not perform its operation.
404         */
405        protected abstract T call(S store) throws StoreException, CommandException;
406    
407        // to do
408        // need to implement on all sub commands and break down the transactions
409    
410        // protected abstract T execute(String id) throws CommandException;
411    
412        /**
413         * Command subclasses must implement this method correct Store can be passed to call(store);
414         *
415         * @return the Store class for use by Callable
416         * @throws CommandException thrown if the command could not perform its operation.
417         */
418        protected abstract Class<? extends Store> getStoreClass();
419    
420        /**
421         * Set the log info with the context of the given coordinator bean.
422         *
423         * @param cBean coordinator bean.
424         */
425        protected void setLogInfo(CoordinatorJobBean cBean) {
426            if (logInfo.getParameter(XLogService.GROUP) == null) {
427                logInfo.setParameter(XLogService.GROUP, cBean.getGroup());
428            }
429            if (logInfo.getParameter(XLogService.USER) == null) {
430                logInfo.setParameter(XLogService.USER, cBean.getUser());
431            }
432            logInfo.setParameter(DagXLogInfoService.JOB, cBean.getId());
433            logInfo.setParameter(DagXLogInfoService.TOKEN, "");
434            logInfo.setParameter(DagXLogInfoService.APP, cBean.getAppName());
435            XLog.Info.get().setParameters(logInfo);
436        }
437    
438        /**
439         * Set the log info with the context of the given coordinator action bean.
440         *
441         * @param action action bean.
442         */
443        protected void setLogInfo(CoordinatorActionBean action) {
444            logInfo.setParameter(DagXLogInfoService.JOB, action.getJobId());
445            // logInfo.setParameter(DagXLogInfoService.TOKEN, action.getLogToken());
446            logInfo.setParameter(DagXLogInfoService.ACTION, action.getId());
447            XLog.Info.get().setParameters(logInfo);
448        }
449    
450        /**
451         * Set the log info with the context of the given workflow bean.
452         *
453         * @param workflow workflow bean.
454         */
455        protected void setLogInfo(WorkflowJobBean workflow) {
456            if (logInfo.getParameter(XLogService.GROUP) == null) {
457                logInfo.setParameter(XLogService.GROUP, workflow.getGroup());
458            }
459            if (logInfo.getParameter(XLogService.USER) == null) {
460                logInfo.setParameter(XLogService.USER, workflow.getUser());
461            }
462            logInfo.setParameter(DagXLogInfoService.JOB, workflow.getId());
463            logInfo.setParameter(DagXLogInfoService.TOKEN, workflow.getLogToken());
464            logInfo.setParameter(DagXLogInfoService.APP, workflow.getAppName());
465            XLog.Info.get().setParameters(logInfo);
466        }
467    
468        /**
469         * Set the log info with the context of the given action bean.
470         *
471         * @param action action bean.
472         */
473        protected void setLogInfo(WorkflowActionBean action) {
474            logInfo.setParameter(DagXLogInfoService.JOB, action.getJobId());
475            logInfo.setParameter(DagXLogInfoService.TOKEN, action.getLogToken());
476            logInfo.setParameter(DagXLogInfoService.ACTION, action.getId());
477            XLog.Info.get().setParameters(logInfo);
478        }
479    
480        /**
481         * Reset the action bean information from the log info.
482         */
483        // TODO check if they are used, else delete
484        protected void resetLogInfoAction() {
485            logInfo.clearParameter(DagXLogInfoService.ACTION);
486            XLog.Info.get().clearParameter(DagXLogInfoService.ACTION);
487        }
488    
489        /**
490         * Reset the workflow bean information from the log info.
491         */
492        // TODO check if they are used, else delete
493        protected void resetLogInfoWorkflow() {
494            logInfo.clearParameter(DagXLogInfoService.JOB);
495            logInfo.clearParameter(DagXLogInfoService.APP);
496            logInfo.clearParameter(DagXLogInfoService.TOKEN);
497            XLog.Info.get().clearParameter(DagXLogInfoService.JOB);
498            XLog.Info.get().clearParameter(DagXLogInfoService.APP);
499            XLog.Info.get().clearParameter(DagXLogInfoService.TOKEN);
500        }
501    
502        /**
503         * Convenience method to increment counters.
504         *
505         * @param group the group name.
506         * @param name the counter name.
507         * @param count increment count.
508         */
509        private void incrCounter(String group, String name, int count) {
510            if (instrumentation != null) {
511                instrumentation.incr(group, name, count);
512            }
513        }
514    
515        /**
516         * Used to increment command counters.
517         *
518         * @param count the increment count.
519         */
520        protected void incrCommandCounter(int count) {
521            incrCounter(INSTRUMENTATION_GROUP, name, count);
522        }
523    
524        /**
525         * Used to increment job counters. The counter name s the same as the command name.
526         *
527         * @param count the increment count.
528         */
529        protected void incrJobCounter(int count) {
530            incrJobCounter(name, count);
531        }
532    
533        /**
534         * Used to increment job counters.
535         *
536         * @param name the job name.
537         * @param count the increment count.
538         */
539        protected void incrJobCounter(String name, int count) {
540            incrCounter(INSTRUMENTATION_JOB_GROUP, name, count);
541        }
542    
543        /**
544         * Return the {@link Instrumentation} instance in use.
545         *
546         * @return the {@link Instrumentation} instance in use.
547         */
548        protected Instrumentation getInstrumentation() {
549            return instrumentation;
550        }
551    
552        /**
553         * Return the identity.
554         *
555         * @return the identity.
556         */
557        @Override
558        public String toString() {
559            StringBuilder sb = new StringBuilder();
560            sb.append(getType());
561            sb.append(",").append(getPriority());
562            return sb.toString();
563        }
564    
565        protected boolean lock(String id) throws InterruptedException {
566            if (id == null || id.length() == 0) {
567                XLog.getLog(getClass()).warn("lock(): Id is null or empty :" + id + ":");
568                return false;
569            }
570            LockToken token = Services.get().get(MemoryLocksService.class).getWriteLock(id, LOCK_TIMEOUT);
571            if (token != null) {
572                locks.add(token);
573                return true;
574            }
575            else {
576                return false;
577            }
578        }
579    
580        /*
581         * TODO - remove store coupling to EM. Store will only contain queries
582         * protected EntityManager getEntityManager() { return
583         * store.getEntityManager(); }
584         */
585        protected T execute(S store) throws CommandException, StoreException {
586            T result = call(store);
587            return result;
588        }
589    }