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 }