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