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 org.apache.oozie.ErrorCode;
018    import org.apache.oozie.FaultInjection;
019    import org.apache.oozie.XException;
020    import org.apache.oozie.service.CallableQueueService;
021    import org.apache.oozie.service.InstrumentationService;
022    import org.apache.oozie.service.MemoryLocksService;
023    import org.apache.oozie.service.Services;
024    import org.apache.oozie.util.Instrumentation;
025    import org.apache.oozie.util.MemoryLocks;
026    import org.apache.oozie.util.XCallable;
027    import org.apache.oozie.util.XLog;
028    
029    import java.util.ArrayList;
030    import java.util.HashMap;
031    import java.util.List;
032    import java.util.Map;
033    import java.util.UUID;
034    
035    /**
036     * Base class for synchronous and asynchronous commands.
037     * <p/>
038     * It enables by API the following pattern:
039     * <p/>
040     * <ul>
041     * <li>single execution: a command instance can be executed only once</li>
042     * <li>eager data loading: loads data for eager precondition check</li>
043     * <li>eager precondition check: verify precondition before obtaining lock</li>
044     * <li>data loading: loads data for precondition check and execution</li>
045     * <li>precondition check: verifies precondition for execution is still met</li>
046     * <li>locking: obtains exclusive lock on key before executing the command</li>
047     * <li>execution: command logic</li>
048     * </ul>
049     * <p/>
050     * It has built in instrumentation and logging.
051     */
052    public abstract class XCommand<T> implements XCallable<T> {
053        public static final String DEFAULT_LOCK_TIMEOUT = "oozie.command.default.lock.timeout";
054    
055        public static final String INSTRUMENTATION_GROUP = "commands";
056    
057        public static final Long DEFAULT_REQUEUE_DELAY = 10L;
058    
059        public XLog LOG = XLog.getLog(getClass());
060    
061        private String key;
062        private String name;
063        private int priority;
064        private String type;
065        private long createdTime;
066        private MemoryLocks.LockToken lock;
067        private boolean used = false;
068    
069        private Map<Long, List<XCommand<?>>> commandQueue;
070        protected boolean dryrun = false;
071        protected Instrumentation instrumentation;
072    
073        protected XLog.Info logInfo;
074    
075        /**
076         * Create a command.
077         *
078         * @param name command name.
079         * @param type command type.
080         * @param priority command priority.
081         */
082        public XCommand(String name, String type, int priority) {
083            this.name = name;
084            this.type = type;
085            this.priority = priority;
086            this.key = name + "_" + UUID.randomUUID();
087            createdTime = System.currentTimeMillis();
088            logInfo = new XLog.Info();
089            instrumentation = Services.get().get(InstrumentationService.class).get();
090        }
091    
092        /**
093         * @param name command name.
094         * @param type command type.
095         * @param priority command priority.
096         * @param dryrun indicates if dryrun option is enabled. if enabled bundle will show a diagnostic output without
097         *        really running the job
098         */
099        public XCommand(String name, String type, int priority, boolean dryrun) {
100            this(name, type, priority);
101            this.dryrun = dryrun;
102        }
103    
104        /**
105         * Return the command name.
106         *
107         * @return the command name.
108         */
109        @Override
110        public String getName() {
111            return name;
112        }
113    
114        /**
115         * Return the callable type.
116         * <p/>
117         * The command type is used for concurrency throttling in the {@link CallableQueueService}.
118         *
119         * @return the command type.
120         */
121        @Override
122        public String getType() {
123            return type;
124        }
125    
126        /**
127         * Return the priority of the command.
128         *
129         * @return the command priority.
130         */
131        @Override
132        public int getPriority() {
133            return priority;
134        }
135    
136        /**
137         * Returns the creation time of the command.
138         *
139         * @return the command creation time, in milliseconds.
140         */
141        @Override
142        public long getCreatedTime() {
143            return createdTime;
144        }
145    
146        /**
147         * Queue a command for execution after the current command execution completes.
148         * <p/>
149         * All commands queued during the execution of the current command will be queued for a single serial execution.
150         * <p/>
151         * If the command execution throws an exception, no command will be effectively queued.
152         *
153         * @param command command to queue.
154         */
155        protected void queue(XCommand<?> command) {
156            queue(command, 0);
157        }
158    
159        /**
160         * Queue a command for delayed execution after the current command execution completes.
161         * <p/>
162         * All commands queued during the execution of the current command with the same delay will be queued for a single
163         * serial execution.
164         * <p/>
165         * If the command execution throws an exception, no command will be effectively queued.
166         *
167         * @param command command to queue.
168         * @param msDelay delay in milliseconds.
169         */
170        protected void queue(XCommand<?> command, long msDelay) {
171            if (commandQueue == null) {
172                commandQueue = new HashMap<Long, List<XCommand<?>>>();
173            }
174            List<XCommand<?>> list = commandQueue.get(msDelay);
175            if (list == null) {
176                list = new ArrayList<XCommand<?>>();
177                commandQueue.put(msDelay, list);
178            }
179            list.add(command);
180        }
181    
182        /**
183         * Obtain an exclusive lock on the {link #getEntityKey}.
184         * <p/>
185         * A timeout of {link #getLockTimeOut} is used when trying to obtain the lock.
186         *
187         * @throws InterruptedException thrown if an interruption happened while trying to obtain the lock
188         * @throws CommandException thrown i the lock could not be obtained.
189         */
190        private void acquireLock() throws InterruptedException, CommandException {
191            lock = Services.get().get(MemoryLocksService.class).getWriteLock(getEntityKey(), getLockTimeOut());
192            if (lock == null) {
193                Instrumentation instrumentation = Services.get().get(InstrumentationService.class).get();
194                instrumentation.incr(INSTRUMENTATION_GROUP, getName() + ".lockTimeOut", 1);
195                if (isReQueueRequired()) {
196                    //if not acquire the lock, re-queue itself with default delay
197                    resetUsed();
198                    queue(this, getRequeueDelay());
199                    LOG.debug("Could not get lock [{0}], timed out [{1}]ms, and requeue itself [{2}]", this.toString(), getLockTimeOut(), getName());
200                } else {
201                    throw new CommandException(ErrorCode.E0606, this.toString(), getLockTimeOut());
202                }
203            } else {
204                LOG.debug("Acquired lock for [{0}] in [{1}]", getEntityKey(), getName());
205            }
206        }
207    
208        /**
209         * Release the lock on the {link #getEntityKey}.
210         */
211        private void releaseLock() {
212            if (lock != null) {
213                lock.release();
214                LOG.debug("Released lock for [{0}] in [{1}]", getEntityKey(), getName());
215            }
216        }
217    
218        /**
219         * Implements the XCommand life-cycle.
220         *
221         * @return the {link #execute} return value.
222         * @throws Exception thrown if the command could not be executed.
223         */
224        @Override
225        public final T call() throws CommandException {
226            if (used) {
227                throw new IllegalStateException(this.getClass().getSimpleName() + " already used.");
228            }
229            used = true;
230            commandQueue = null;
231            Instrumentation instrumentation = Services.get().get(InstrumentationService.class).get();
232            instrumentation.incr(INSTRUMENTATION_GROUP, getName() + ".executions", 1);
233            Instrumentation.Cron callCron = new Instrumentation.Cron();
234            try {
235                callCron.start();
236                eagerLoadState();
237                LOG = XLog.resetPrefix(LOG);
238                eagerVerifyPrecondition();
239                try {
240                    T ret = null;
241                    if (isLockRequired()) {
242                        Instrumentation.Cron acquireLockCron = new Instrumentation.Cron();
243                        acquireLockCron.start();
244                        acquireLock();
245                        acquireLockCron.stop();
246                        instrumentation.addCron(INSTRUMENTATION_GROUP, getName() + ".acquireLock", acquireLockCron);
247                    }
248                    if (!isLockRequired() || (isLockRequired() && lock != null)) {
249                        LOG.debug("Load state for [{0}]", getEntityKey());
250                        loadState();
251                        LOG = XLog.resetPrefix(LOG);
252                        LOG.debug("Precondition check for command [{0}] key [{1}]", getName(), getEntityKey());
253                        verifyPrecondition();
254                        LOG.debug("Execute command [{0}] key [{1}]", getName(), getEntityKey());
255                        Instrumentation.Cron executeCron = new Instrumentation.Cron();
256                        executeCron.start();
257                        ret = execute();
258                        executeCron.stop();
259                        instrumentation.addCron(INSTRUMENTATION_GROUP, getName() + ".execute", executeCron);
260                    }
261                    if (commandQueue != null) {
262                        CallableQueueService callableQueueService = Services.get().get(CallableQueueService.class);
263                        for (Map.Entry<Long, List<XCommand<?>>> entry : commandQueue.entrySet()) {
264                            LOG.debug("Queuing [{0}] commands with delay [{1}]ms", entry.getValue().size(), entry.getKey());
265                            if (!callableQueueService.queueSerial(entry.getValue(), entry.getKey())) {
266                                LOG.warn("Could not queue [{0}] commands with delay [{1}]ms, queue full", entry.getValue()
267                                        .size(), entry.getKey());
268                            }
269                        }
270                    }
271                    return ret;
272                }
273                finally {
274                    if (isLockRequired()) {
275                        releaseLock();
276                    }
277                }
278            }
279            catch(PreconditionException pex){
280                LOG.warn(pex.getMessage().toString() + ", Error Code: " + pex.getErrorCode().toString(), pex);
281                instrumentation.incr(INSTRUMENTATION_GROUP, getName() + ".preconditionfailed", 1);
282                return null;
283            }
284            catch (XException ex) {
285                LOG.error("XException, ", ex);
286                instrumentation.incr(INSTRUMENTATION_GROUP, getName() + ".xexceptions", 1);
287                if (ex instanceof CommandException) {
288                    throw (CommandException) ex;
289                }
290                else {
291                    throw new CommandException(ex);
292                }
293            }
294            catch (Exception ex) {
295                LOG.error("Exception, ", ex);
296                instrumentation.incr(INSTRUMENTATION_GROUP, getName() + ".exceptions", 1);
297                throw new CommandException(ErrorCode.E0607, ex);
298            }
299            finally {
300                FaultInjection.deactivate("org.apache.oozie.command.SkipCommitFaultInjection");
301                callCron.stop();
302                instrumentation.addCron(INSTRUMENTATION_GROUP, getName() + ".call", callCron);
303            }
304        }
305    
306        /**
307         * Return the time out when acquiring a lock.
308         * <p/>
309         * The value is loaded from the Oozie configuration, the property {link #DEFAULT_LOCK_TIMEOUT}.
310         * <p/>
311         * Subclasses should override this method if they want to use a different time out.
312         *
313         * @return the lock time out in milliseconds.
314         */
315        protected long getLockTimeOut() {
316            return Services.get().getConf().getLong(DEFAULT_LOCK_TIMEOUT, 5 * 1000);
317        }
318    
319        /**
320         * Indicate if the the command requires locking.
321         * <p/>
322         * Subclasses should override this method if they require locking.
323         *
324         * @return <code>true/false</code>
325         */
326        protected abstract boolean isLockRequired();
327    
328        /**
329         * Return the entity key for the command.
330         * <p/>
331         *
332         * @return the entity key for the command.
333         */
334        protected abstract String getEntityKey();
335    
336        /**
337         * Indicate if the the command requires to requeue itself if the lock is not acquired.
338         * <p/>
339         * Subclasses should override this method if they don't want to requeue.
340         * <p/>
341         * Default is true.
342         *
343         * @return <code>true/false</code>
344         */
345        protected boolean isReQueueRequired() {
346            return true;
347        }
348    
349        /**
350         * Load the necessary state to perform an eager precondition check.
351         * <p/>
352         * This implementation does a NOP.
353         * <p/>
354         * Subclasses should override this method and load the state needed to do an eager precondition check.
355         * <p/>
356         * A trivial implementation is calling {link #loadState}.
357         */
358        protected void eagerLoadState() throws CommandException{
359        }
360    
361        /**
362         * Verify the precondition for the command before obtaining a lock.
363         * <p/>
364         * This implementation does a NOP.
365         * <p/>
366         * A trivial implementation is calling {link #verifyPrecondition}.
367         *
368         * @throws CommandException thrown if the precondition is not met.
369         */
370        protected void eagerVerifyPrecondition() throws CommandException,PreconditionException {
371        }
372    
373        /**
374         * Load the necessary state to perform the precondition check and to execute the command.
375         * <p/>
376         * Subclasses must implement this method and load the state needed to do the precondition check and execute the
377         * command.
378         */
379        protected abstract void loadState() throws CommandException;
380    
381        /**
382         * Verify the precondition for the command after a lock has been obtain, just before executing the command.
383         * <p/>
384         *
385         * @throws CommandException thrown if the precondition is not met.
386         */
387        protected abstract void verifyPrecondition() throws CommandException,PreconditionException;
388    
389        /**
390         * Command execution body.
391         * <p/>
392         * This method will be invoked after the {link #loadState} and {link #verifyPrecondition} methods.
393         * <p/>
394         * If the command requires locking, this method will be invoked ONLY if the lock has been acquired.
395         *
396         * @return a return value from the execution of the command, only meaningful if the command is executed
397         *         synchronously.
398         * @throws CommandException thrown if the command execution failed.
399         */
400        protected abstract T execute() throws CommandException;
401    
402    
403        /**
404         * Return the {@link Instrumentation} instance in use.
405         *
406         * @return the {@link Instrumentation} instance in use.
407         */
408        protected Instrumentation getInstrumentation() {
409            return instrumentation;
410        }
411    
412        /**
413         * @param used set false to the used
414         */
415        public void resetUsed() {
416            this.used = false;
417        }
418    
419    
420        /**
421         * Return the delay time for requeue
422         *
423         * @return delay time when requeue itself
424         */
425        protected Long getRequeueDelay() {
426            return DEFAULT_REQUEUE_DELAY;
427        }
428    
429        /**
430         * Get command key
431         *
432         * @return command key
433         */
434        @Override
435        public String getKey(){
436            return this.key;
437        }
438    
439        /**
440         * Get XLog log
441         *
442         * @return XLog
443         */
444        public XLog getLog() {
445            return LOG;
446        }
447    
448    }