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