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