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.service;
016    
017    import java.util.ArrayList;
018    import java.util.Date;
019    import java.util.HashMap;
020    import java.util.HashSet;
021    import java.util.List;
022    import java.util.Map;
023    import java.util.Set;
024    import java.util.Map.Entry;
025    import java.util.concurrent.BlockingQueue;
026    import java.util.concurrent.ConcurrentHashMap;
027    import java.util.concurrent.RejectedExecutionException;
028    import java.util.concurrent.ThreadPoolExecutor;
029    import java.util.concurrent.TimeUnit;
030    import java.util.concurrent.atomic.AtomicInteger;
031    import java.util.concurrent.atomic.AtomicLong;
032    
033    import org.apache.hadoop.conf.Configuration;
034    import org.apache.oozie.client.OozieClient.SYSTEM_MODE;
035    import org.apache.oozie.util.Instrumentable;
036    import org.apache.oozie.util.Instrumentation;
037    import org.apache.oozie.util.PriorityDelayQueue;
038    import org.apache.oozie.util.XCallable;
039    import org.apache.oozie.util.XLog;
040    import org.apache.oozie.util.PriorityDelayQueue.QueueElement;
041    
042    /**
043     * The callable queue service queues {@link XCallable}s for asynchronous execution.
044     * <p/>
045     * Callables can be queued for immediate execution or for delayed execution (some time in the future).
046     * <p/>
047     * Callables are consumed from the queue for execution based on their priority.
048     * <p/>
049     * When the queues (for immediate execution and for delayed execution) are full, the callable queue service stops
050     * queuing callables.
051     * <p/>
052     * A thread-pool is used to execute the callables asynchronously.
053     * <p/>
054     * The following configuration parameters control the callable queue service:
055     * <p/>
056     * {@link #CONF_QUEUE_SIZE} size of the immediate execution queue. Defaulf value is 10000.
057     * <p/>
058     * {@link #CONF_THREADS} number of threads in the thread-pool used for asynchronous command execution. When this number
059     * of threads is reached, commands remain the queue until threads become available. Sets up a priority queue for the
060     * execution of Commands via a ThreadPool. Sets up a Delayed Queue to handle actions which will be ready for execution
061     * sometime in the future.
062     */
063    public class CallableQueueService implements Service, Instrumentable {
064        private static final String INSTRUMENTATION_GROUP = "callablequeue";
065        private static final String INSTR_IN_QUEUE_TIME_TIMER = "time.in.queue";
066        private static final String INSTR_EXECUTED_COUNTER = "executed";
067        private static final String INSTR_FAILED_COUNTER = "failed";
068        private static final String INSTR_QUEUED_COUNTER = "queued";
069        private static final String INSTR_QUEUE_SIZE_SAMPLER = "queue.size";
070        private static final String INSTR_THREADS_ACTIVE_SAMPLER = "threads.active";
071    
072        public static final String CONF_PREFIX = Service.CONF_PREFIX + "CallableQueueService.";
073    
074        public static final String CONF_QUEUE_SIZE = CONF_PREFIX + "queue.size";
075        public static final String CONF_THREADS = CONF_PREFIX + "threads";
076        public static final String CONF_CALLABLE_CONCURRENCY = CONF_PREFIX + "callable.concurrency";
077    
078        public static final int CONCURRENCY_DELAY = 500;
079    
080        public static final int SAFE_MODE_DELAY = 60000;
081    
082        private final Map<String, AtomicInteger> activeCallables = new HashMap<String, AtomicInteger>();
083    
084        private final Map<String, Date> uniqueCallables = new ConcurrentHashMap<String, Date>();
085    
086        private int maxCallableConcurrency;
087    
088        private boolean callableBegin(XCallable<?> callable) {
089            synchronized (activeCallables) {
090                AtomicInteger counter = activeCallables.get(callable.getType());
091                if (counter == null) {
092                    counter = new AtomicInteger(1);
093                    activeCallables.put(callable.getType(), counter);
094                    return true;
095                }
096                else {
097                    int i = counter.incrementAndGet();
098                    return i <= maxCallableConcurrency;
099                }
100            }
101        }
102    
103        private void callableEnd(XCallable<?> callable) {
104            synchronized (activeCallables) {
105                AtomicInteger counter = activeCallables.get(callable.getType());
106                if (counter == null) {
107                    throw new IllegalStateException("It should not happen");
108                }
109                else {
110                    counter.decrementAndGet();
111                }
112            }
113        }
114    
115        // Callables are wrapped with the this wrapper for execution, for logging
116        // and instrumentation.
117        // The wrapper implements Runnable and Comparable to be able to work with an
118        // executor and a priority queue.
119        class CallableWrapper extends PriorityDelayQueue.QueueElement<XCallable<?>> implements Runnable {
120            private Instrumentation.Cron cron;
121    
122            public CallableWrapper(XCallable<?> callable, long delay) {
123                super(callable, callable.getPriority(), delay, TimeUnit.MILLISECONDS);
124                cron = new Instrumentation.Cron();
125                cron.start();
126            }
127    
128            public void run() {
129                if (Services.get().getSystemMode() == SYSTEM_MODE.SAFEMODE) {
130                    log.info("Oozie is in SAFEMODE, requeuing callable [{0}] with [{1}]ms delay", getElement().getType(),
131                            SAFE_MODE_DELAY);
132                    setDelay(SAFE_MODE_DELAY, TimeUnit.MILLISECONDS);
133                    removeFromUniqueCallables();
134                    queue(this, true);
135                    return;
136                }
137                XCallable<?> callable = getElement();
138                try {
139                    if (callableBegin(callable)) {
140                        cron.stop();
141                        addInQueueCron(cron);
142                        XLog.Info.get().clear();
143                        XLog log = XLog.getLog(getClass());
144                        log.trace("executing callable [{0}]", callable.getName());
145    
146                        removeFromUniqueCallables();
147                        try {
148                            callable.call();
149                            incrCounter(INSTR_EXECUTED_COUNTER, 1);
150                            log.trace("executed callable [{0}]", callable.getName());
151                        }
152                        catch (Exception ex) {
153                            incrCounter(INSTR_FAILED_COUNTER, 1);
154                            log.warn("exception callable [{0}], {1}", callable.getName(), ex.getMessage(), ex);
155                        }
156                        finally {
157                            XLog.Info.get().clear();
158                        }
159                    }
160                    else {
161                        log.warn("max concurrency for callable [{0}] exceeded, requeueing with [{1}]ms delay", callable
162                                .getType(), CONCURRENCY_DELAY);
163                        setDelay(CONCURRENCY_DELAY, TimeUnit.MILLISECONDS);
164                        removeFromUniqueCallables();
165                        queue(this, true);
166                        incrCounter(callable.getType() + "#exceeded.concurrency", 1);
167                    }
168                }
169                finally {
170                    callableEnd(callable);
171                }
172            }
173    
174            /**
175             * @return String the queue dump
176             */
177            @Override
178            public String toString() {
179                return "delay=" + getDelay(TimeUnit.MILLISECONDS) + ", elements=" + getElement().toString();
180            }
181    
182            /**
183             * Filter the duplicate callables from the list before queue this.
184             * <p/>
185             * If it is single callable, checking if key is in unique map or not.
186             * <p/>
187             * If it is composite callable, remove duplicates callables from the composite.
188             *
189             * @return true if this callable should be queued
190             */
191            public boolean filterDuplicates() {
192                XCallable<?> callable = getElement();
193                if (callable instanceof CompositeCallable) {
194                    return ((CompositeCallable) callable).removeDuplicates();
195                }
196                else {
197                    return uniqueCallables.containsKey(callable.getKey()) == false;
198                }
199            }
200    
201            /**
202             * Add the keys to the set
203             */
204            public void addToUniqueCallables() {
205                XCallable<?> callable = getElement();
206                if (callable instanceof CompositeCallable) {
207                    ((CompositeCallable) callable).addToUniqueCallables();
208                }
209                else {
210                    ((ConcurrentHashMap<String, Date>) uniqueCallables).putIfAbsent(callable.getKey(), new Date());
211                }
212            }
213    
214            /**
215             * Remove the keys from the set
216             */
217            public void removeFromUniqueCallables() {
218                XCallable<?> callable = getElement();
219                if (callable instanceof CompositeCallable) {
220                    ((CompositeCallable) callable).removeFromUniqueCallables();
221                }
222                else {
223                    uniqueCallables.remove(callable.getKey());
224                }
225            }
226    
227        }
228    
229        class CompositeCallable implements XCallable<Void> {
230            private List<XCallable<?>> callables;
231            private String name;
232            private int priority;
233            private long createdTime;
234    
235            public CompositeCallable(List<? extends XCallable<?>> callables) {
236                this.callables = new ArrayList<XCallable<?>>(callables);
237                priority = 0;
238                createdTime = Long.MAX_VALUE;
239                StringBuilder sb = new StringBuilder();
240                String separator = "[";
241                for (XCallable<?> callable : callables) {
242                    priority = Math.max(priority, callable.getPriority());
243                    createdTime = Math.min(createdTime, callable.getCreatedTime());
244                    sb.append(separator).append(callable.getName());
245                    separator = ",";
246                }
247                sb.append("]");
248                name = sb.toString();
249            }
250    
251            @Override
252            public String getName() {
253                return name;
254            }
255    
256            @Override
257            public String getType() {
258                return "#composite#" + callables.get(0).getType();
259            }
260    
261            @Override
262            public String getKey() {
263                return "#composite#" + callables.get(0).getKey();
264            }
265    
266            @Override
267            public int getPriority() {
268                return priority;
269            }
270    
271            @Override
272            public long getCreatedTime() {
273                return createdTime;
274            }
275    
276            public Void call() throws Exception {
277                XLog log = XLog.getLog(getClass());
278    
279                for (XCallable<?> callable : callables) {
280                    log.trace("executing callable [{0}]", callable.getName());
281                    try {
282                        callable.call();
283                        incrCounter(INSTR_EXECUTED_COUNTER, 1);
284                        log.trace("executed callable [{0}]", callable.getName());
285                    }
286                    catch (Exception ex) {
287                        incrCounter(INSTR_FAILED_COUNTER, 1);
288                        log.warn("exception callable [{0}], {1}", callable.getName(), ex.getMessage(), ex);
289                    }
290                }
291    
292                // ticking -1 not to count the call to the composite callable
293                incrCounter(INSTR_EXECUTED_COUNTER, -1);
294                return null;
295            }
296    
297            /*
298             * (non-Javadoc)
299             *
300             * @see java.lang.Object#toString()
301             */
302            @Override
303            public String toString() {
304                if (callables.size() == 0) {
305                    return null;
306                }
307                StringBuilder sb = new StringBuilder();
308                int size = callables.size();
309                for (int i = 0; i < size; i++) {
310                    XCallable<?> callable = callables.get(i);
311                    sb.append("(");
312                    sb.append(callable.toString());
313                    if (i + 1 == size) {
314                        sb.append(")");
315                    }
316                    else {
317                        sb.append("),");
318                    }
319                }
320                return sb.toString();
321            }
322    
323            /**
324             * Remove the duplicate callables from the list before queue them
325             *
326             * @return true if callables should be queued
327             */
328            public boolean removeDuplicates() {
329                Set<String> set = new HashSet<String>();
330                List<XCallable<?>> filteredCallables = new ArrayList<XCallable<?>>();
331                if (callables.size() == 0) {
332                    return false;
333                }
334                for (XCallable<?> callable : callables) {
335                    if (!uniqueCallables.containsKey(callable.getKey()) && !set.contains(callable.getKey())) {
336                        filteredCallables.add(callable);
337                        set.add(callable.getKey());
338                    }
339                }
340                callables = filteredCallables;
341                if (callables.size() == 0) {
342                    return false;
343                }
344                return true;
345            }
346    
347            /**
348             * Add the keys to the set
349             */
350            public void addToUniqueCallables() {
351                for (XCallable<?> callable : callables) {
352                    ((ConcurrentHashMap<String, Date>) uniqueCallables).putIfAbsent(callable.getKey(), new Date());
353                }
354            }
355    
356            /**
357             * Remove the keys from the set
358             */
359            public void removeFromUniqueCallables() {
360                for (XCallable<?> callable : callables) {
361                    uniqueCallables.remove(callable.getKey());
362                }
363            }
364    
365        }
366    
367        private XLog log = XLog.getLog(getClass());
368    
369        private int queueSize;
370        private PriorityDelayQueue<CallableWrapper> queue;
371        private AtomicLong delayQueueExecCounter = new AtomicLong(0);
372        private ThreadPoolExecutor executor;
373        private Instrumentation instrumentation;
374    
375        /**
376         * Convenience method for instrumentation counters.
377         *
378         * @param name counter name.
379         * @param count count to increment the counter.
380         */
381        private void incrCounter(String name, int count) {
382            if (instrumentation != null) {
383                instrumentation.incr(INSTRUMENTATION_GROUP, name, count);
384            }
385        }
386    
387        private void addInQueueCron(Instrumentation.Cron cron) {
388            if (instrumentation != null) {
389                instrumentation.addCron(INSTRUMENTATION_GROUP, INSTR_IN_QUEUE_TIME_TIMER, cron);
390            }
391        }
392    
393        /**
394         * Initialize the command queue service.
395         *
396         * @param services services instance.
397         */
398        @Override
399        @SuppressWarnings("unchecked")
400        public void init(Services services) {
401            Configuration conf = services.getConf();
402    
403            queueSize = conf.getInt(CONF_QUEUE_SIZE, 10000);
404            int threads = conf.getInt(CONF_THREADS, 10);
405    
406            queue = new PriorityDelayQueue<CallableWrapper>(3, 1000 * 30, TimeUnit.MILLISECONDS, queueSize) {
407                @Override
408                protected void debug(String msgTemplate, Object... msgArgs) {
409                    log.trace(msgTemplate, msgArgs);
410                }
411            };
412    
413            // IMPORTANT: The ThreadPoolExecutor does not always the execute
414            // commands out of the queue, there are
415            // certain conditions where commands are pushed directly to a thread.
416            // As we are using a queue with DELAYED semantics (i.e. execute the
417            // command in 5 mins) we need to make
418            // sure that the commands are always pushed to the queue.
419            // To achieve this (by looking a the ThreadPoolExecutor.execute()
420            // implementation, we are making the pool
421            // minimum size equals to the maximum size (thus threads are keep always
422            // running) and we are warming up
423            // all those threads (the for loop that runs dummy runnables).
424            executor = new ThreadPoolExecutor(threads, threads, 10, TimeUnit.SECONDS, (BlockingQueue) queue);
425    
426            for (int i = 0; i < threads; i++) {
427                executor.execute(new Runnable() {
428                    public void run() {
429                        try {
430                            Thread.sleep(100);
431                        }
432                        catch (InterruptedException ex) {
433                            log.warn("Could not warm up threadpool {0}", ex.getMessage(), ex);
434                        }
435                    }
436                });
437            }
438    
439            maxCallableConcurrency = conf.getInt(CONF_CALLABLE_CONCURRENCY, 3);
440        }
441    
442        /**
443         * Destroy the command queue service.
444         */
445        @Override
446        public void destroy() {
447            try {
448                long limit = System.currentTimeMillis() + 30 * 1000;// 30 seconds
449                executor.shutdown();
450                queue.clear();
451                while (!executor.awaitTermination(1000, TimeUnit.MILLISECONDS)) {
452                    log.info("Waiting for executor to shutdown");
453                    if (System.currentTimeMillis() > limit) {
454                        log.warn("Gave up, continuing without waiting for executor to shutdown");
455                        break;
456                    }
457                }
458            }
459            catch (InterruptedException ex) {
460                log.warn(ex);
461            }
462        }
463    
464        /**
465         * Return the public interface for command queue service.
466         *
467         * @return {@link CallableQueueService}.
468         */
469        @Override
470        public Class<? extends Service> getInterface() {
471            return CallableQueueService.class;
472        }
473    
474        /**
475         * @return int size of queue
476         */
477        public synchronized int queueSize() {
478            return queue.size();
479        }
480    
481        private synchronized boolean queue(CallableWrapper wrapper, boolean ignoreQueueSize) {
482            if (!ignoreQueueSize && queue.size() >= queueSize) {
483                log.warn("queue if full, ignoring queuing for [{0}]", wrapper.getElement());
484                return false;
485            }
486            if (!executor.isShutdown()) {
487                if (wrapper.filterDuplicates()) {
488                    wrapper.addToUniqueCallables();
489                    try {
490                        executor.execute(wrapper);
491                    }
492                    catch (RejectedExecutionException ree) {
493                        wrapper.removeFromUniqueCallables();
494                        throw ree;
495                    }
496                }
497            }
498            else {
499                log.warn("Executor shutting down, ignoring queueing of [{0}]", wrapper.getElement());
500            }
501            return true;
502        }
503    
504        /**
505         * Queue a callable for asynchronous execution.
506         *
507         * @param callable callable to queue.
508         * @return <code>true</code> if the callable was queued, <code>false</code> if the queue is full and the callable
509         *         was not queued.
510         */
511        public boolean queue(XCallable<?> callable) {
512            return queue(callable, 0);
513        }
514    
515        /**
516         * Queue a list of callables for serial execution.
517         * <p/>
518         * Useful to serialize callables that may compete with each other for resources.
519         * <p/>
520         * All callables will be processed with the priority of the highest priority of all callables.
521         *
522         * @param callables callables to be executed by the composite callable.
523         * @return <code>true</code> if the callables were queued, <code>false</code> if the queue is full and the callables
524         *         were not queued.
525         */
526        @SuppressWarnings("unchecked")
527        public boolean queueSerial(List<? extends XCallable<?>> callables) {
528            return queueSerial(callables, 0);
529        }
530    
531        /**
532         * Queue a callable for asynchronous execution sometime in the future.
533         *
534         * @param callable callable to queue for delayed execution
535         * @param delay time, in milliseconds, that the callable should be delayed.
536         * @return <code>true</code> if the callable was queued, <code>false</code> if the queue is full and the callable
537         *         was not queued.
538         */
539        public synchronized boolean queue(XCallable<?> callable, long delay) {
540            if (callable == null) {
541                return true;
542            }
543            boolean queued = false;
544            if (Services.get().getSystemMode() == SYSTEM_MODE.SAFEMODE) {
545                log.warn("[queue] System is in SAFEMODE. Hence no callable is queued. current queue size " + queue.size());
546            }
547            else {
548                queued = queue(new CallableWrapper(callable, delay), false);
549                if (queued) {
550                    incrCounter(INSTR_QUEUED_COUNTER, 1);
551                }
552                else {
553                    log.warn("Could not queue callable");
554                }
555            }
556            return queued;
557        }
558    
559        /**
560         * Queue a list of callables for serial execution sometime in the future.
561         * <p/>
562         * Useful to serialize callables that may compete with each other for resources.
563         * <p/>
564         * All callables will be processed with the priority of the highest priority of all callables.
565         *
566         * @param callables callables to be executed by the composite callable.
567         * @param delay time, in milliseconds, that the callable should be delayed.
568         * @return <code>true</code> if the callables were queued, <code>false</code> if the queue is full and the callables
569         *         were not queued.
570         */
571        @SuppressWarnings("unchecked")
572        public synchronized boolean queueSerial(List<? extends XCallable<?>> callables, long delay) {
573            boolean queued;
574            if (callables == null || callables.size() == 0) {
575                queued = true;
576            }
577            else if (callables.size() == 1) {
578                queued = queue(callables.get(0), delay);
579            }
580            else {
581                XCallable<?> callable = new CompositeCallable(callables);
582                queued = queue(callable, delay);
583                if (queued) {
584                    incrCounter(INSTR_QUEUED_COUNTER, callables.size());
585                }
586            }
587            return queued;
588        }
589    
590        /**
591         * Instruments the callable queue service.
592         *
593         * @param instr instance to instrument the callable queue service to.
594         */
595        public void instrument(Instrumentation instr) {
596            instrumentation = instr;
597            instr.addSampler(INSTRUMENTATION_GROUP, INSTR_QUEUE_SIZE_SAMPLER, 60, 1, new Instrumentation.Variable<Long>() {
598                public Long getValue() {
599                    return (long) queue.size();
600                }
601            });
602            instr.addSampler(INSTRUMENTATION_GROUP, INSTR_THREADS_ACTIVE_SAMPLER, 60, 1,
603                    new Instrumentation.Variable<Long>() {
604                        public Long getValue() {
605                            return (long) executor.getActiveCount();
606                        }
607                    });
608        }
609    
610        /**
611         * Get the list of strings of queue dump
612         *
613         * @return the list of string that representing each CallableWrapper
614         */
615        public List<String> getQueueDump() {
616            List<String> list = new ArrayList<String>();
617            for (QueueElement<CallableWrapper> qe : queue) {
618                if (qe.toString() == null) {
619                    continue;
620                }
621                list.add(qe.toString());
622            }
623            return list;
624        }
625    
626        /**
627         * Get the list of strings of uniqueness map dump
628         *
629         * @return the list of string that representing the key of each command in the queue
630         */
631        public List<String> getUniqueDump() {
632            List<String> list = new ArrayList<String>();
633            for (Entry<String, Date> entry : uniqueCallables.entrySet()) {
634                list.add(entry.toString());
635            }
636            return list;
637        }
638    
639    }