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 }